diff --git a/.coveragerc b/.coveragerc index ac399fe492..90a8c361a3 100644 --- a/.coveragerc +++ b/.coveragerc @@ -1,10 +1,13 @@ [run] include = + calico/etcddriver/* calico/felix/* calico/openstack/* calico/*.py omit = + calico/test/* calico/felix/test/* calico/openstack/test/* + calico/etcddriver/test/* branch = True concurrency = eventlet diff --git a/CHANGES.md b/CHANGES.md index 16f4176731..57c8a21b5b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,8 @@ # Changelog +- Felix now parses the etcd snapshot in parallel with the event stream; + this dramatically increases scale when under load. + ## 1.2.0 - Add liveness reporting to Felix. Felix now reports its liveness into diff --git a/calico/common.py b/calico/common.py index 580a7d2bd7..3afed8950d 100644 --- a/calico/common.py +++ b/calico/common.py @@ -22,23 +22,24 @@ Calico common utilities. """ import errno -import gevent -import gevent.local -import itertools import logging import logging.handlers -import netaddr -import netaddr.core import os import re import sys from types import StringTypes + +import netaddr +import netaddr.core from netaddr.strategy import eui48 _log = logging.getLogger(__name__) AGENT_TYPE_CALICO = 'Calico agent' -FORMAT_STRING = '%(asctime)s [%(levelname)s][%(process)s/%(tid)d] %(name)s %(lineno)d: %(message)s' + +FORMAT_STRING = '%(asctime)s [%(levelname)s][%(process)s/%(thread)d] %(name)s %(lineno)d: %(message)s' +# Used "tid", which we swap for the greenlet ID, instead of "thread" +FORMAT_STRING_GEVENT = '%(asctime)s [%(levelname)s][%(process)s/%(tid)d] %(name)s %(lineno)d: %(message)s' # This format string deliberately uses two different styles of format # specifier. The %()s form is used by the logging module: the {} form is used @@ -89,24 +90,6 @@ VALID_IPAM_POOL_ID_RE = re.compile(r'^[0-9\.:a-fA-F\-]{1,43}$') EXPECTED_IPAM_POOL_KEYS = set(["cidr", "masquerade"]) -tid_storage = gevent.local.local() -tid_counter = itertools.count() -# Ought to do itertools.count(start=1), but python 2.6 does not support it. -tid_counter.next() - -def greenlet_id(): - """ - Returns an integer greenlet ID. - itertools.count() is atomic, if the internet is correct. - http://stackoverflow.com/questions/23547604/python-counter-atomic-increment - """ - try: - tid = tid_storage.tid - except: - tid = tid_counter.next() - tid_storage.tid = tid - return tid - def validate_port(port): """ @@ -178,18 +161,13 @@ def mkdir_p(path): except TypeError: try: os.makedirs(path) - except OSError as exc: # Python >2.5 + except OSError as exc: # Python >2.5 if exc.errno == errno.EEXIST and os.path.isdir(path): pass else: raise -class GreenletFilter(logging.Filter): - def filter(self, record): - record.tid = greenlet_id() - return True - -def default_logging(): +def default_logging(gevent_in_use=True, syslog_executable_name=None): """ Sets up the Calico default logging, with default severities. @@ -210,7 +188,7 @@ def default_logging(): root_logger = logging.getLogger() root_logger.setLevel(logging.DEBUG) - executable_name = os.path.basename(sys.argv[0]) + executable_name = syslog_executable_name or os.path.basename(sys.argv[0]) syslog_format = SYSLOG_FORMAT_STRING.format(excname=executable_name) syslog_formatter = logging.Formatter(syslog_format) if os.path.exists("/dev/log"): @@ -223,18 +201,22 @@ def default_logging(): root_logger.addHandler(syslog_handler) - file_formatter = logging.Formatter(FORMAT_STRING) + format_string = FORMAT_STRING_GEVENT if gevent_in_use else FORMAT_STRING + file_formatter = logging.Formatter(format_string) stream_handler = logging.StreamHandler(sys.stdout) stream_handler.setLevel(logging.ERROR) stream_handler.setFormatter(file_formatter) - stream_handler.addFilter(GreenletFilter()) + if gevent_in_use: + from geventutils import GreenletFilter + stream_handler.addFilter(GreenletFilter()) root_logger.addHandler(stream_handler) def complete_logging(logfile=None, file_level=logging.DEBUG, syslog_level=logging.ERROR, - stream_level=logging.ERROR): + stream_level=logging.ERROR, + gevent_in_use=True): """ Updates the logging configuration based on learned configuration. @@ -279,9 +261,13 @@ def complete_logging(logfile=None, if logfile and file_level is not None: if not file_handler: mkdir_p(os.path.dirname(logfile)) - formatter = logging.Formatter(FORMAT_STRING) + format_string = (FORMAT_STRING_GEVENT if gevent_in_use + else FORMAT_STRING) + formatter = logging.Formatter(format_string) file_handler = logging.handlers.WatchedFileHandler(logfile) - file_handler.addFilter(GreenletFilter()) + if gevent_in_use: + from geventutils import GreenletFilter + file_handler.addFilter(GreenletFilter()) file_handler.setLevel(file_level) file_handler.setFormatter(formatter) root_logger.addHandler(file_handler) @@ -402,6 +388,7 @@ def validate_endpoint(config, combined_id, endpoint): if issues: raise ValidationFailed(" ".join(issues)) + def validate_rules(profile_id, rules): """ Ensures that the supplied rules are valid. Once this routine has returned @@ -588,6 +575,7 @@ def validate_tags(profile_id, tags): if issues: raise ValidationFailed(" ".join(issues)) + def validate_ipam_pool(pool_id, pool, ip_version): """ Validates and canonicalises an IPAM pool dict. Removes any fields that diff --git a/calico/datamodel_v1.py b/calico/datamodel_v1.py index 77c51499a4..e9b40720e5 100644 --- a/calico/datamodel_v1.py +++ b/calico/datamodel_v1.py @@ -190,6 +190,12 @@ def path_for_status(self): def __str__(self): return self.__class__.__name__ + ("<%s>" % self.endpoint) + def __repr__(self): + return self.__class__.__name__ + ("(%r,%r,%r,%r)" % (self.host, + self.orchestrator, + self.workload, + self.endpoint)) + def __eq__(self, other): if other is self: return True diff --git a/calico/etcddriver/__init__.py b/calico/etcddriver/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/calico/etcddriver/__main__.py b/calico/etcddriver/__main__.py new file mode 100644 index 0000000000..f5714fafe0 --- /dev/null +++ b/calico/etcddriver/__main__.py @@ -0,0 +1,59 @@ +# -*- coding: utf-8 -*- + +# Copyright (c) 2014, 2015 Metaswitch Networks +# All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. You may obtain +# a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +# License for the specific language governing permissions and limitations +# under the License. + +""" +calico.etcddriver.__main__ +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Main entry point for the etcd driver, responsible for basic logging config +and starting our threads. +""" + +import logging +import os +import socket +import sys + +from calico.etcddriver import driver +from calico import common + +_log = logging.getLogger(__name__) + +last_ppid = os.getppid() +common.default_logging(gevent_in_use=False, + syslog_executable_name="calico-felix-etcd") + +felix_sck = socket.socket(socket.AF_UNIX, + socket.SOCK_STREAM) +try: + felix_sck.connect(sys.argv[1]) +except: + _log.exception("Failed to connect to Felix") + raise + +etcd_driver = driver.EtcdDriver(felix_sck) +etcd_driver.start() + +while not etcd_driver.join(timeout=1): + parent_pid = os.getppid() + # Defensive, just in case we don't get a socket error, check if the + # parent PID has changed, indicating that Felix has died. + if parent_pid == 1 or parent_pid != last_ppid: + _log.critical("Process adopted, assuming felix has died") + etcd_driver.stop() + break +_log.critical("Driver shutting down.") diff --git a/calico/etcddriver/driver.py b/calico/etcddriver/driver.py new file mode 100644 index 0000000000..07278b6a1d --- /dev/null +++ b/calico/etcddriver/driver.py @@ -0,0 +1,837 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.etcddriver.driver +~~~~~~~~~~~~~~~~~~~~~~~~ + +Contains the logic for the etcd driver process, which monitors etcd for +changes and sends them to Felix over a unix socket. + +The driver is responsible for + +* loading the configuration from etcd at start-of-day (Felix needs this before + it can receive further updates) +* handling the initial load of data from etcd +* watching etcd for changes +* doing the above in parallel and merging the result into a consistent + sequence of events +* resolving directory deletions so that if a directory is deleted, it tells + Felix about all the individual keys that are deleted. +""" +from functools import partial +from httplib import HTTPException +import logging +from Queue import Queue, Empty +import socket + +from ijson import JSONError + +try: + # simplejson is a faster drop-in replacement. + import simplejson as json +except ImportError: + import json +from threading import Thread, Event +import time +from urlparse import urlparse + +from ijson.backends import yajl2 as ijson +import urllib3 +from urllib3 import HTTPConnectionPool +from urllib3.exceptions import ReadTimeoutError + +from calico.etcddriver.protocol import ( + MessageReader, MSG_TYPE_INIT, MSG_TYPE_CONFIG, MSG_TYPE_RESYNC, + MSG_KEY_ETCD_URL, MSG_KEY_HOSTNAME, MSG_KEY_LOG_FILE, MSG_KEY_SEV_FILE, + MSG_KEY_SEV_SYSLOG, MSG_KEY_SEV_SCREEN, STATUS_WAIT_FOR_READY, + STATUS_RESYNC, STATUS_IN_SYNC, MSG_TYPE_CONFIG_LOADED, + MSG_KEY_GLOBAL_CONFIG, MSG_KEY_HOST_CONFIG, MSG_TYPE_UPDATE, MSG_KEY_KEY, + MSG_KEY_VALUE, MessageWriter, MSG_TYPE_STATUS, MSG_KEY_STATUS, + WriteFailed) +from calico.etcdutils import ACTION_MAPPING +from calico.common import complete_logging +from calico.monotonic import monotonic_time +from calico.datamodel_v1 import ( + READY_KEY, CONFIG_DIR, dir_for_per_host_config, VERSION_DIR, + ROOT_DIR) +from calico.etcddriver.hwm import HighWaterTracker + +_log = logging.getLogger(__name__) + + +# Bound on the size of the queue between watcher and resync thread. In +# general, Felix and the resync thread process much more quickly than the +# watcher can read from etcd so this is defensive. +WATCHER_QUEUE_SIZE = 20000 + + +class EtcdDriver(object): + def __init__(self, felix_sck): + # Wrap the socket with our protocol reader/writer objects. + self._msg_reader = MessageReader(felix_sck) + self._msg_writer = MessageWriter(felix_sck) + + # Global stop event used to signal to all threads to stop. + self._stop_event = Event() + + # Threads to own the connection from/to Felix. The resync thread + # is responsible for doing resyncs and merging updates from the + # watcher thread (which it manages). + self._reader_thread = Thread(target=self._read_from_socket, + name="reader-thread") + self._reader_thread.daemon = True + self._resync_thread = Thread(target=self._resync_and_merge, + name="resync-thread") + self._resync_thread.daemon = True + self._watcher_thread = None # Created on demand + self._watcher_stop_event = None + self._watcher_start_index = None + + # High-water mark cache. Owned by resync thread. + self._hwms = HighWaterTracker() + self._first_resync = True + self._resync_http_pool = None + self._cluster_id = None + + # Set by the reader thread once the init message has been received + # from Felix. + self._init_received = Event() + # Initial config, received in the init message. + self._etcd_base_url = None + self._hostname = None + # Set by the reader thread once the logging config has been received + # from Felix. Triggers the first resync. + self._config_received = Event() + + # Flag to request a resync. Set by the reader thread, polled by the + # resync and merge thread. + self._resync_requested = False + + def start(self): + """Starts the driver's reader and resync threads.""" + self._reader_thread.start() + self._resync_thread.start() + + def join(self, timeout=None): + """ + Blocks until the driver stops or until timeout expires. + + :returns True if the driver stopped, False on timeout. + """ + self._stop_event.wait(timeout=timeout) + stopped = self._stop_event.is_set() + if stopped: + self._resync_thread.join(timeout=timeout) + resync_alive = self._resync_thread.is_alive() + stopped &= not resync_alive + _log.debug("Resync thread alive: %s", resync_alive) + + self._reader_thread.join(timeout=timeout) + reader_alive = self._reader_thread.is_alive() + stopped &= not reader_alive + _log.debug("Reader thread alive: %s", reader_alive) + + try: + self._watcher_thread.join(timeout=timeout) + watcher_alive = self._watcher_thread.is_alive() + stopped &= not watcher_alive + _log.debug("Watcher thread alive: %s", watcher_alive) + except AttributeError: + pass + return stopped + + def stop(self): + _log.info("Stopping driver") + self._stop_event.set() + + def _read_from_socket(self): + """ + Thread: reader thread. Reads messages from Felix and fans them out. + """ + try: + while not self._stop_event.is_set(): + for msg_type, msg in self._msg_reader.new_messages(timeout=1): + if msg_type == MSG_TYPE_INIT: + # Init message, received at start of day. + self._handle_init(msg) + elif msg_type == MSG_TYPE_CONFIG: + # Config message, expected after we send the raw + # config to Felix. + self._handle_config(msg) + elif msg_type == MSG_TYPE_RESYNC: + # Request to do a resync. + self._handle_resync(msg) + else: + _log.error("Unexpected message from Felix: %s", msg) + raise RuntimeError("Unexpected message from Felix") + finally: + _log.error("Reader thread shutting down, triggering stop event") + self.stop() + + def _handle_init(self, msg): + """ + Handle init message from Felix. + + Called from the reader thread. + """ + # OK to dump the msg, it's a one-off. + _log.info("Got init message from Felix %s", msg) + self._etcd_base_url = msg[MSG_KEY_ETCD_URL].rstrip("/") + self._etcd_url_parts = urlparse(self._etcd_base_url) + self._hostname = msg[MSG_KEY_HOSTNAME] + self._init_received.set() + + def _handle_config(self, msg): + """ + Handle config message from Felix. + + Called from the reader thread. + """ + complete_logging(msg[MSG_KEY_LOG_FILE], + file_level=msg[MSG_KEY_SEV_FILE], + syslog_level=msg[MSG_KEY_SEV_SYSLOG], + stream_level=msg[MSG_KEY_SEV_SCREEN], + gevent_in_use=False) + self._config_received.set() + _log.info("Received config from Felix: %s", msg) + + def _handle_resync(self, msg): + _log.info("Got resync message from felix: %s", msg) + self._resync_requested = True + + def _resync_and_merge(self): + """ + Thread: Resync-and-merge thread. Loads the etcd snapshot, merges + it with the events going on concurrently and sends the event stream + to Felix. + """ + _log.info("Resync thread started, waiting for config to be loaded...") + self._init_received.wait() + _log.info("Config loaded; continuing.") + + while not self._stop_event.is_set(): + loop_start = monotonic_time() + try: + # Start with a fresh HTTP pool just in case it got into a bad + # state. + self._resync_http_pool = self.get_etcd_connection() + # Before we get to the snapshot, Felix needs the configuration. + self._send_status(STATUS_WAIT_FOR_READY) + self._wait_for_ready() + self._preload_config() + # Wait for config if we have not already received it. + self._wait_for_config() + # Kick off the snapshot request as far as the headers. + self._send_status(STATUS_RESYNC) + resp, snapshot_index = self._start_snapshot_request() + # Before reading from the snapshot, start the watcher thread. + self._ensure_watcher_running(snapshot_index) + # Incrementally process the snapshot, merging in events from + # the queue. + self._process_snapshot_and_events(resp, snapshot_index) + # We're now in-sync. Tell Felix. + self._send_status(STATUS_IN_SYNC) + # Then switch to processing events only. + self._process_events_only() + except WriteFailed: + _log.exception("Write to Felix failed; shutting down.") + self.stop() + except WatcherDied: + _log.warning("Watcher died; resyncing.") + self._stop_watcher() # Clean up the event + except (urllib3.exceptions.HTTPError, + HTTPException, + socket.error) as e: + _log.error("Request to etcd failed: %r; resyncing.", e) + self._stop_watcher() + if monotonic_time() - loop_start < 1: + _log.warning("May be tight looping, sleeping...") + time.sleep(1) + except ResyncRequested: + _log.info("Resync requested, looping to start a new resync. " + "Leaving watcher running if possible.") + except ResyncRequired: + _log.warn("Detected inconsistency requiring a full resync, " + "stopping watcher") + self._stop_watcher() + except DriverShutdown: + _log.info("Driver shut down.") + return + except: + _log.exception("Unexpected exception; shutting down.") + self.stop() + raise + finally: + self._first_resync = False + self._resync_requested = False + + def _wait_for_config(self): + while not self._config_received.is_set(): + _log.info("Waiting for Felix to process the config...") + self._check_stop_event() + self._config_received.wait(1) + _log.info("Felix sent us the config, continuing.") + + def _wait_for_ready(self): + """ + Waits for the global Ready flag to be set. We don't load the first + snapshot until that flag is set. + """ + ready = False + while not ready and not self._stop_event.is_set(): + # Read failure here will be handled by outer loop. + resp = self._etcd_request(self._resync_http_pool, READY_KEY) + try: + etcd_resp = json.loads(resp.data) + ready = etcd_resp["node"]["value"] == "true" + mod_idx = etcd_resp["node"]["modifiedIndex"] + except (TypeError, ValueError, KeyError) as e: + _log.warning("Failed to load Ready flag from etcd: %r", e) + time.sleep(1) + else: + _log.info("Ready flag set to %s", etcd_resp["node"]["value"]) + self._hwms.update_hwm(READY_KEY, mod_idx) + self._check_stop_event() + + def _check_stop_event(self): + if self._stop_event.is_set(): + _log.info("Told to stop, raising DriverShutdown.") + raise DriverShutdown() + + def _preload_config(self): + """ + Loads the config for Felix from etcd and sends it to Felix as a + dedicated message. + """ + _log.info("Pre-loading config.") + global_config = self._load_config(CONFIG_DIR) + host_config_dir = dir_for_per_host_config(self._hostname) + host_config = self._load_config(host_config_dir) + self._msg_writer.send_message( + MSG_TYPE_CONFIG_LOADED, + { + MSG_KEY_GLOBAL_CONFIG: global_config, + MSG_KEY_HOST_CONFIG: host_config, + } + ) + _log.info("Sent config message to Felix.") + + def _load_config(self, config_dir): + """ + Loads all the config keys from the given etcd directory. + """ + # Read failure here will be handled by outer loop. + resp = self._etcd_request(self._resync_http_pool, + config_dir, recursive=True) + try: + etcd_resp = json.loads(resp.data) + if etcd_resp.get("errorCode") == 100: # Not found + _log.info("No config found at %s", config_dir) + return {} + config_nodes = etcd_resp["node"]["nodes"] + config = {} + for node in config_nodes: + if "key" in node and "value" in node: + config[node["key"].split("/")[-1]] = node["value"] + except (TypeError, ValueError, KeyError) as e: + _log.warning("Failed to load config from etcd: %r," + "data %r", e, resp.data) + raise ResyncRequired(e) + return config + + def _start_snapshot_request(self): + """ + Issues the HTTP request to etcd to load the snapshot but only + loads it as far as the headers. + :return: tuple of response and snapshot's etcd index. + :raises HTTPException + :raises HTTPError + :raises socket.error + :raises DriverShutdown if the etcd cluster ID changes. + """ + _log.info("Loading snapshot headers...") + resp = self._etcd_request(self._resync_http_pool, + VERSION_DIR, + recursive=True, + timeout=120, + preload_content=False) + snapshot_index = int(resp.getheader("x-etcd-index", 1)) + if not self._cluster_id: + _log.error("Snapshot response did not contain cluster ID, " + "resyncing to avoid inconsistency") + raise ResyncRequired() + _log.info("Got snapshot headers, snapshot index is %s; starting " + "watcher...", snapshot_index) + return resp, snapshot_index + + def _etcd_request(self, http_pool, key, timeout=5, wait_index=None, + recursive=False, preload_content=None): + """ + Make a request to etcd on the given HTTP pool for the given key + and check the cluster ID. + + :param timeout: Read timeout for the request. + :param int wait_index: If set, issues a watch request. + :param recursive: True to request a recursive GET or watch. + + :return: The urllib3 Response object. + """ + resp = self._issue_etcd_request( + http_pool, key, timeout, wait_index, + recursive, preload_content + ) + self._check_cluster_id(resp) + return resp + + def _issue_etcd_request(self, http_pool, key, timeout=5, wait_index=None, + recursive=False, preload_content=None): + fields = {} + if recursive: + _log.debug("Adding recursive=true to request") + fields["recursive"] = "true" + if wait_index is not None: + _log.debug("Request is a watch, adding wait* headers and forcing " + "preload_content to False") + fields["wait"] = "true" + fields["waitIndex"] = wait_index + preload_content = False + if preload_content is None: + preload_content = True + resp = http_pool.request( + "GET", + self._calculate_url(key), + fields=fields or None, + timeout=timeout, + preload_content=preload_content + ) + return resp + + def _check_cluster_id(self, resp): + """ + Checks the x-etcd-cluster-id header for changes since the last call. + + On change, stops the driver and raises DriverShutdown. + :param resp: urllib3 Response object. + """ + cluster_id = resp.getheader("x-etcd-cluster-id") + if cluster_id: + if self._cluster_id: + if self._cluster_id != cluster_id: + _log.error("etcd cluster ID changed from %s to %s. " + "This invalidates our local state so Felix " + "must restart.", self._cluster_id, cluster_id) + self.stop() + raise DriverShutdown() + else: + _log.info("First successful read from etcd. Cluster ID: %s", + cluster_id) + self._cluster_id = cluster_id + else: + # Missing on certain error responses. + _log.warning("etcd response was missing cluster ID header, unable " + "to check cluster ID") + + def _process_snapshot_and_events(self, etcd_response, snapshot_index): + """ + Processes the etcd snapshot response incrementally while, concurrently, + merging in updates from the watcher thread. + :param etcd_response: file-like object representing the etcd response. + :param snapshot_index: the etcd index of the response. + """ + self._hwms.start_tracking_deletions() + parse_snapshot(etcd_response, + callback=partial(self._handle_etcd_node, + snapshot_index=snapshot_index)) + + # Save occupancy by throwing away the deletion tracking metadata. + self._hwms.stop_tracking_deletions() + # Scan for deletions that happened before the snapshot. We effectively + # mark all the values seen in the current snapshot above and then this + # sweeps the ones we didn't touch. + self._scan_for_deletions(snapshot_index) + + def _handle_etcd_node(self, snap_mod, snap_key, snap_value, + snapshot_index=None): + """ + Callback for use with parse_snapshot. Called once for each key/value + pair that is found. + + Handles the key/value itself and then checks for work from the + watcher. + + :param snap_mod: Modified index of the key. + :param snap_key: The key itself. + :param snap_value: The value attached to the key. + :param snapshot_index: Index of the snapshot as a whole. + """ + assert snapshot_index is not None + old_hwm = self._hwms.update_hwm(snap_key, snapshot_index) + if snap_mod > old_hwm: + # This specific key's HWM is newer than the previous + # version we've seen, send an update. + self._on_key_updated(snap_key, snap_value) + # After we process an update from the snapshot, process several + # updates from the watcher queue (if there are any). We limit the + # number to ensure that we always finish the snapshot eventually. + # The limit isn't too sensitive but values much lower than 100 seemed + # to starve the watcher in testing. + for _ in xrange(100): + if not self._watcher_queue or self._watcher_queue.empty(): + # Don't block on the watcher if there's nothing to do. + break + try: + self._handle_next_watcher_event(resync_in_progress=True) + except WatcherDied: + # Continue processing to ensure that we make + # progress. + _log.warning("Watcher thread died, continuing " + "with snapshot") + break + self._check_stop_event() + + def _process_events_only(self): + """ + Loops processing the event stream from the watcher thread and feeding + it to etcd. + :raises WatcherDied: + :raises FelixWriteFailed: + :raises DriverShutdown: + """ + _log.info("In sync, now processing events only...") + while not self._stop_event.is_set(): + self._handle_next_watcher_event(resync_in_progress=False) + self._msg_writer.flush() + self._check_stop_event() + + def _scan_for_deletions(self, snapshot_index): + """ + Scans the high-water mark cache for keys that haven't been seen since + before the snapshot_index and deletes them. + """ + if self._first_resync: + _log.info("First resync: skipping deletion scan") + return + # Find any keys that were deleted while we were unable to + # keep up with etcd. + _log.info("Scanning for deletions") + deleted_keys = self._hwms.remove_old_keys(snapshot_index) + for ev_key in deleted_keys: + # We didn't see the value during the snapshot or via + # the event queue. It must have been deleted. + self._on_key_updated(ev_key, None) + _log.info("Found %d deleted keys", len(deleted_keys)) + + def _handle_next_watcher_event(self, resync_in_progress): + """ + Waits for an event on the watcher queue and sends it to Felix. + :raises DriverShutdown: + :raises WatcherDied: + :raises FelixWriteFailed: + :raises ResyncRequested: + """ + if self._watcher_queue is None: + raise WatcherDied() + while not self._stop_event.is_set(): + # To make sure we always make progress, only trigger a new resync + # if we're not in the middle of one. + if not resync_in_progress and self._resync_requested: + _log.info("Resync requested, triggering one.") + raise ResyncRequested() + try: + event = self._watcher_queue.get(timeout=1) + except Empty: + pass + else: + break + else: + raise DriverShutdown() + if event is None: + self._watcher_queue = None + raise WatcherDied() + ev_mod, ev_key, ev_val = event + if ev_val is not None: + # Normal update. + self._hwms.update_hwm(ev_key, ev_mod) + self._on_key_updated(ev_key, ev_val) + else: + # Deletion. In case this is a directory deletion, we search the + # trie for anything that is under the deleted key and send + # individual deletions to Felix for each one. + deleted_keys = self._hwms.store_deletion(ev_key, + ev_mod) + for child_key in deleted_keys: + self._on_key_updated(child_key, None) + + def _ensure_watcher_running(self, snapshot_index): + """ + Starts a new watcher from the given snapshot index, if needed. + """ + if (self._watcher_thread is not None and + self._watcher_thread.is_alive() and + self._watcher_stop_event is not None and + not self._watcher_stop_event.is_set() and + self._watcher_queue is not None and + self._watcher_start_index <= snapshot_index): + _log.info("Watcher is still alive and started from a valid index, " + "leaving it running") + return + + self._watcher_start_index = snapshot_index + self._watcher_queue = Queue(maxsize=WATCHER_QUEUE_SIZE) + self._watcher_stop_event = Event() + # Note: we pass the queue and event in as arguments so that the thread + # will always access the current queue and event. If it used self.xyz + # to access them then an old thread that is shutting down could access + # a new queue. + self._watcher_thread = Thread(target=self.watch_etcd, + args=(snapshot_index + 1, + self._watcher_queue, + self._watcher_stop_event), + name="watcher-thread") + self._watcher_thread.daemon = True + self._watcher_thread.start() + + def _stop_watcher(self): + """ + If it's running, signals the watcher thread to stop. + """ + if self._watcher_stop_event is not None: + _log.info("Watcher was running before, stopping it") + self._watcher_stop_event.set() + self._watcher_stop_event = None + + def get_etcd_connection(self): + return HTTPConnectionPool(self._etcd_url_parts.hostname, + self._etcd_url_parts.port or 2379, + maxsize=1) + + def _on_key_updated(self, key, value): + """ + Called when we've worked out that a key has been updated/deleted. + + Does any local processing and sends the update to Felix. + :param str key: The etcd key that has changed. + :param str|NoneType value: the new value of the key (None indicates + deletion). + """ + if key == READY_KEY and value != "true": + # Special case: the global Ready flag has been unset, trigger a + # resync, which will poll the Ready flag until it is set to true + # again. + _log.warning("Ready key no longer set to true, triggering resync.") + raise ResyncRequired() + self._msg_writer.send_message( + MSG_TYPE_UPDATE, + { + MSG_KEY_KEY: key, + MSG_KEY_VALUE: value, + }, + flush=False + ) + + def _send_status(self, status): + """ + Queues the given status to felix as a status message. + """ + _log.info("Sending status to Felix: %s", status) + self._msg_writer.send_message( + MSG_TYPE_STATUS, + { + MSG_KEY_STATUS: status, + } + ) + + def _calculate_url(self, etcd_key): + return self._etcd_base_url + "/v2/keys/" + etcd_key.strip("/") + + def watch_etcd(self, next_index, event_queue, stop_event): + """ + Thread: etcd watcher thread. Watches etcd for changes and + sends them over the queue to the resync thread, which owns + the socket to Felix. + + Dies if it receives an error from etcd. + + Note: it is important that we pass the index, queue and event + as parameters to ensure that each watcher thread only touches + the versions of those values that were created for it as + opposed to a later-created watcher thread. + + :param next_index: The etcd index to start watching from. + :param event_queue: Queue of updates back to the resync thread. + :param stop_event: Event used to stop this thread when it is no + longer needed. + """ + _log.info("Watcher thread started") + http = None + try: + while not self._stop_event.is_set() and not stop_event.is_set(): + if not http: + _log.info("No HTTP pool, creating one...") + http = self.get_etcd_connection() + try: + _log.debug("Waiting on etcd index %s", next_index) + resp = self._etcd_request(http, + VERSION_DIR, + recursive=True, + wait_index=next_index, + timeout=90) + if resp.status != 200: + _log.warning("etcd watch returned bad HTTP status: %s", + resp.status) + self._check_cluster_id(resp) + resp_body = resp.data # Force read inside try block. + except ReadTimeoutError: + _log.debug("Watch read timed out, restarting watch at " + "index %s", next_index) + # Workaround urllib3 bug #718. After a ReadTimeout, the + # connection is incorrectly recycled. + http = None + continue + try: + etcd_resp = json.loads(resp_body) + if "errorCode" in etcd_resp: + _log.error("Error from etcd: %s; triggering a resync.", + etcd_resp) + break + node = etcd_resp["node"] + key = node["key"] + action = ACTION_MAPPING[etcd_resp["action"]] + is_dir = node.get("dir", False) + value = node.get("value") + if is_dir: + if action == "delete": + if key.rstrip("/") in (VERSION_DIR, ROOT_DIR): + # Special case: if the whole keyspace is + # deleted, that implies the ready flag is gone + # too. Break out of the loop to trigger a + # resync. This avoids queuing up a bunch of + # events that would be discarded by the + # resync thread. + _log.warning("Whole %s deleted, resyncing", + VERSION_DIR) + break + else: + # Just ignore sets to directories, we only track + # leaves. + _log.debug("Skipping non-delete to dir %s", key) + continue + modified_index = node["modifiedIndex"] + except (KeyError, TypeError, ValueError): + _log.exception("Unexpected format for etcd response: %r;" + "triggering a resync.", + resp_body) + break + else: + event_queue.put((modified_index, key, value)) + next_index = modified_index + 1 + except: + _log.exception("Exception finishing watcher thread.") + raise + finally: + # Signal to the resync thread that we've exited. + _log.info("Watcher thread finished. Signalling to resync thread.") + event_queue.put(None) + + +def parse_snapshot(resp, callback): + """ + Iteratively parses the response to the etcd snapshot, calling the + callback with each key/value pair found. + + :raises ResyncRequired if the snapshot contains an error response. + """ + _log.debug("Parsing snapshot response...") + if resp.status != 200: + raise ResyncRequired("Read from etcd failed. HTTP status code %s", + resp.status) + parser = ijson.parse(resp) # urllib3 response is file-like. + + try: + prefix, event, value = next(parser) + _log.debug("Read first token from response %s, %s, %s", prefix, event, + value) + if event == "start_map": + # As expected, response is a map. + _parse_map(parser, callback) + else: + _log.error("Response from etcd did non contain a JSON map.") + raise ResyncRequired("Bad response from etcd") + except JSONError: + _log.exception("Response from etcd containers bad JSON.") + raise ResyncRequired("Bad JSON from etcd") + + +def _parse_map(parser, callback): + """ + Searches the stream of JSON tokens for key/value pairs. + + Calls itself recursively to handle subdirectories. + + :param parser: iterator, returning JSON parse event tuples. + :param callback: callback to call when a key/value pair is found. + """ + # Expect a sequence of keys and values terminated by an "end_map" event. + mod_index = None + node_key = None + node_value = None + while True: + prefix, event, value = next(parser) + _log.debug("Parsing %s, %s, %s", prefix, event, value) + if event == "map_key": + map_key = value + prefix, event, value = next(parser) + if map_key == "modifiedIndex": + mod_index = value + elif map_key == "key": + node_key = value + elif map_key == "value": + node_value = value + elif map_key == "errorCode": + raise ResyncRequired("Error from etcd, etcd error code %s", + value) + elif map_key == "nodes": + while True: + prefix, event, value = next(parser) + if event == "start_map": + _parse_map(parser, callback) + elif event == "end_array": + break + else: + raise ValueError("Unexpected: %s" % event) + else: + assert event == "end_map", ("Unexpected JSON event %s %s %s" % + (prefix, event, value)) + if (node_key is not None and + node_value is not None and + mod_index is not None): + callback(mod_index, node_key, node_value) + break + + +class WatcherDied(Exception): + pass + + +class DriverShutdown(Exception): + pass + + +class ResyncRequired(Exception): + pass + + +class ResyncRequested(Exception): + pass diff --git a/calico/etcddriver/hwm.py b/calico/etcddriver/hwm.py new file mode 100644 index 0000000000..21da89bce2 --- /dev/null +++ b/calico/etcddriver/hwm.py @@ -0,0 +1,218 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.etcddriver.hwm +~~~~~~~~~~~~~~~~~~~~~ + +The HighWaterTracker is used to resolve the high water mark for each etcd +key when processing a snapshot and event stream in parallel. +""" + +import logging +import re +import string + +from datrie import Trie +import datrie +import urllib + +_log = logging.getLogger(__name__) + +# The trie implementation that we use requires us to specify the character set +# in advance... +# Symbols that are allowed in our etcd keys. +TRIE_SYMBOLS = "/_-:." +# Chars we allow in the trie. In addition to alphanumerics and our +# white-listed symbols, we also use % for %-encoding of unexpected symbols. +TRIE_CHARS = string.ascii_letters + string.digits + TRIE_SYMBOLS + "%" +# Regex that matches chars that are allowed in the trie. +TRIE_CHARS_MATCH = re.compile(r'^[%s]+$' % re.escape(TRIE_CHARS)) + + +class HighWaterTracker(object): + """ + Tracks the highest etcd index for which we've seen a particular + etcd key. + + This class is expected to be used as follows: + + Starting with a resync, while also merging events from our watch on etcd: + + * Call start_tracking_deletions() to enable resolution between events + and the snapshot. + * Repeatedly call update_hwm() and store_deletion(), feeding in the + data from the snapshot and event stream. + * At the end of the snapshot processing, call stop_tracking_deletions() + to discard the tracking metadata (which would otherwise grow + indefinitely). + * Call remove_old_keys() to find and delete any keys that have not been + seen since before the snapshot was started, and hence must have been + deleted before the snapshot was taken. + + While in sync: + + * feed in events with update_hwm() and store_deletion(). + + At any point, if a new resync is required restart from + "Call start_tracking_deletions()..." + + """ + def __init__(self): + # We use a trie to track the highest etcd index at which we've seen + # each key. The trie implementation forces a fixed character set; + # we explicitly allow the characters we expect and encode any others + # that we're not expecting. + self._hwms = Trie(TRIE_CHARS) + + # Set to a Trie while we're tracking deletions. None otherwise. + self._deletion_hwms = None + # Optimization: tracks the highest etcd index at which we've seen a + # deletion. This allows us to skip an expensive lookup in the + # _deletion_hwms trie for events that come after the deletion. + self._latest_deletion = None + + def start_tracking_deletions(self): + """ + Starts tracking which subtrees have been deleted so that update_hwm + can skip updates to keys that have subsequently been deleted. + + Should be paired with a call to stop_tracking_deletions() to release + the associated tracking data structures. + """ + _log.info("Started tracking deletions") + self._deletion_hwms = Trie(TRIE_CHARS) + self._latest_deletion = None + + def stop_tracking_deletions(self): + """ + Stops deletion tracking and frees up the associated resources. + + Calling this asserts that subsequent calls to update_hwm() will only + use HWMs after any stored deletes. + """ + _log.info("Stopped tracking deletions") + self._deletion_hwms = None + self._latest_deletion = None + + def update_hwm(self, key, new_mod_idx): + """ + Updates the HWM for a key if the new value is greater than the old. + If deletion tracking is enabled, resolves deletions so that updates + to subtrees that have been deleted are skipped iff the deletion is + after the update in HWM order. + + :return int|NoneType: the old HWM of the key (or the HWM at which it + was deleted) or None if it did not previously exist. + """ + _log.debug("Updating HWM for %s to %s", key, new_mod_idx) + key = encode_key(key) + if (self._deletion_hwms is not None and + # Optimization: avoid expensive lookup if this update comes + # after all deletions. + new_mod_idx < self._latest_deletion): + # We're tracking deletions, check that this key hasn't been + # deleted. + del_hwm = self._deletion_hwms.longest_prefix_value(key, None) + if new_mod_idx < del_hwm: + _log.debug("Key %s previously deleted, skipping", key) + return del_hwm + try: + old_hwm = self._hwms[key] # Trie doesn't have get(). + except KeyError: + old_hwm = None + if old_hwm < new_mod_idx: # Works for None too. + _log.debug("Key %s HWM updated to %s, previous %s", + key, new_mod_idx, old_hwm) + self._hwms[key] = new_mod_idx + return old_hwm + + def store_deletion(self, key, deletion_mod_idx): + """ + Store that a given key (or directory) was deleted at a given HWM. + :return: List of known keys that were deleted. This will be the + leaves only when a subtree is being deleted. + """ + _log.debug("Key %s deleted", key) + key = encode_key(key) + self._latest_deletion = max(deletion_mod_idx, self._latest_deletion) + if self._deletion_hwms is not None: + _log.debug("Tracking deletion in deletions trie") + self._deletion_hwms[key] = deletion_mod_idx + deleted_keys = [] + for child_key, child_mod in self._hwms.items(key): + del self._hwms[child_key] + deleted_keys.append(decode_key(child_key)) + _log.debug("Found %s keys deleted under %s", len(deleted_keys), key) + return deleted_keys + + def remove_old_keys(self, hwm_limit): + """ + Deletes and returns all keys that have HWMs less than hwm_limit. + :return: list of keys that were deleted. + """ + assert not self._deletion_hwms, \ + "Delete tracking incompatible with remove_old_keys()" + _log.info("Removing keys that are older than %s", hwm_limit) + old_keys = [] + state = datrie.State(self._hwms) + state.walk(u"") + it = datrie.Iterator(state) + while it.next(): + value = it.data() + if value < hwm_limit: + old_keys.append(it.key()) + for old_key in old_keys: + del self._hwms[old_key] + _log.info("Deleted %s old keys", len(old_keys)) + return map(decode_key, old_keys) + + def __len__(self): + return len(self._hwms) + + +def encode_key(key): + """ + Encode an etcd key for use in the trie. + + This does three things: + * Encodes any characters that are not supported by the trie using + %-encoding. + * Adds a trailing slash if not present. This prevents /foobar/baz from + being seen as a subtree of /foo/. + * Converts the result to a unicode string, which is what is required + by the trie. + + Since our datamodel specifies the characters that are allowed, the first + operation should be a no-op on most keys but it's better to be tolerant + here than to blow up. + """ + if key[-1] != "/": + suffixed_key = key + "/" + else: + suffixed_key = key + encoded_key = unicode(urllib.quote(suffixed_key.encode("utf8"), + safe=TRIE_SYMBOLS)) + assert TRIE_CHARS_MATCH.match(encoded_key), ( + "Key %r encoded to %r contained invalid chars" % (key, encoded_key) + ) + return encoded_key + + +def decode_key(key): + """ + Reverses the encoding done by encode_key. + """ + key = urllib.unquote(key.encode("utf8")).decode("utf8") + return key[:-1] diff --git a/calico/etcddriver/protocol.py b/calico/etcddriver/protocol.py new file mode 100644 index 0000000000..5fa5a80b3e --- /dev/null +++ b/calico/etcddriver/protocol.py @@ -0,0 +1,176 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.etcddriver.protocol +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Protocol constants for Felix <-> Driver protocol. +""" +import logging +import socket +import errno +from io import BytesIO +import msgpack +import select + +_log = logging.getLogger(__name__) + +MSG_KEY_TYPE = "type" + +# Init message Felix -> Driver. +MSG_TYPE_INIT = "init" +MSG_KEY_ETCD_URL = "etcd_url" +MSG_KEY_HOSTNAME = "hostname" + +# Config loaded message Driver -> Felix. +MSG_TYPE_CONFIG_LOADED = "config_loaded" +MSG_KEY_GLOBAL_CONFIG = "global" +MSG_KEY_HOST_CONFIG = "host" + +# Config message Felix -> Driver. +MSG_TYPE_CONFIG = "conf" +MSG_KEY_LOG_FILE = "log_file" +MSG_KEY_SEV_FILE = "sev_file" +MSG_KEY_SEV_SCREEN = "sev_screen" +MSG_KEY_SEV_SYSLOG = "sev_syslog" + +# Status message Driver -> Felix. +MSG_TYPE_STATUS = "stat" +MSG_KEY_STATUS = "status" +STATUS_WAIT_FOR_READY = "wait-for-ready" +STATUS_RESYNC = "resync" +STATUS_IN_SYNC = "in-sync" + +# Force resync message Felix->Driver. +MSG_TYPE_RESYNC = "resync" + +# Update message Driver -> Felix. +MSG_TYPE_UPDATE = "u" +MSG_KEY_KEY = "k" +MSG_KEY_VALUE = "v" + + +FLUSH_THRESHOLD = 200 + + +class SocketClosed(Exception): + """The socket was unexpectedly closed by the other end.""" + pass + + +class WriteFailed(Exception): + """Write to the socket failed.""" + pass + + +class MessageWriter(object): + """ + Wrapper around a socket used to write protocol messages. + + Supports buffering a number of messages for subsequent flush(). + """ + def __init__(self, sck): + self._sck = sck + self._buf = BytesIO() + self._updates_pending = 0 + + def send_message(self, msg_type, fields=None, flush=True): + """ + Send a message of the given type with the given fields. + Optionally, flush the data to the socket. + + This method will flush the buffer if it grows too large in any + case. + + :param msg_type: one of the MSG_TYPE_* constants. + :param dict fields: dict mapping MSG_KEY_* constants to values. + :param flush: True to force the data to be written immediately. + """ + msg = {MSG_KEY_TYPE: msg_type} + if fields: + msg.update(fields) + self._buf.write(msgpack.dumps(msg)) + if flush: + self.flush() + else: + self._maybe_flush() + + def _maybe_flush(self): + self._updates_pending += 1 + if self._updates_pending > FLUSH_THRESHOLD: + self.flush() + + def flush(self): + """ + Flushes the write buffer to the socket immediately. + """ + _log.debug("Flushing the buffer to the socket") + buf_contents = self._buf.getvalue() + if buf_contents: + try: + self._sck.sendall(buf_contents) + except socket.error as e: + _log.exception("Failed to write to socket") + raise WriteFailed(e) + self._buf = BytesIO() + self._updates_pending = 0 + + +class MessageReader(object): + def __init__(self, sck): + self._sck = sck + self._unpacker = msgpack.Unpacker() + + def new_messages(self, timeout=1): + """ + Generator: generates 0 or more tuples containing message type and + message body (as a dict). + + May generate 0 events in certain conditions even if there are + events available. (If the socket returns EAGAIN, for example.) + + :param timeout: Maximum time to block waiting on the socket before + giving up. No exception is raised upon timeout but 0 events + are generated. + :raises SocketClosed if the socket is closed. + :raises socket.error if an unexpected socket error occurs. + """ + if timeout is not None: + read_ready, _, _ = select.select([self._sck], [], [], timeout) + if not read_ready: + return + try: + data = self._sck.recv(16384) + except socket.error as e: + if e.errno in (errno.EAGAIN, + errno.EWOULDBLOCK, + errno.EINTR): + _log.debug("Retryable error on read.") + return + else: + _log.error("Failed to read from socket: %r", e) + raise + if not data: + # No data indicates an orderly shutdown of the socket, + # which shouldn't happen. + _log.error("Socket closed by other end.") + raise SocketClosed() + # Feed the data into the Unpacker, if it has enough data it will then + # generate some messages. + self._unpacker.feed(data) + for msg in self._unpacker: + _log.debug("Unpacked message: %s", msg) + # coverage.py doesn't fully support yield statements. + yield msg[MSG_KEY_TYPE], msg # pragma: nocover diff --git a/calico/etcddriver/test/__init__.py b/calico/etcddriver/test/__init__.py new file mode 100644 index 0000000000..edc7577a0c --- /dev/null +++ b/calico/etcddriver/test/__init__.py @@ -0,0 +1,6 @@ +# Copyright (c) Metaswitch Networks 2015. All rights reserved. + +import logging + +_log = logging.getLogger(__name__) + diff --git a/calico/etcddriver/test/stubs.py b/calico/etcddriver/test/stubs.py new file mode 100644 index 0000000000..f43575e71d --- /dev/null +++ b/calico/etcddriver/test/stubs.py @@ -0,0 +1,369 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.etcddriver.test.stubs +~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Stub objects used for testing driver/protocol code. +""" +import json +import threading + +import logging +from Queue import Queue, Empty + +from calico.etcddriver.protocol import ( + MessageReader, MessageWriter, MSG_KEY_TYPE +) + +_log = logging.getLogger(__name__) + + +# Singleton representing a flush in the stream of writes. +class Sigil(object): + def __init__(self, name): + self.name = name + + def __str__(self): + return "<%s>" % self.name + + +FLUSH = Sigil("FLUSH") + + +class StubMessageReader(MessageReader): + """ + Replacement for the Driver's MessageReader, which is how it reads + from Felix. + + Allows us to send messages as if we were Felix. + """ + def __init__(self, sck): + super(StubMessageReader, self).__init__(sck) + self.queue = Queue() + + def send_msg(self, msg_type, fields=None): + """Called by the test to send the driver a message.""" + msg = { + MSG_KEY_TYPE: msg_type + } + msg.update(fields or {}) + self.queue.put((msg_type, msg)) + + def send_timeout(self): + """Called by the test to send the driver a timeout.""" + self.queue.put(None) + + def send_exception(self, exc): + """Called by the test to raise an exception from the driver's read.""" + self.queue.put(exc) + + def new_messages(self, timeout=None): + """Called by the driver to receive new messages.""" + while True: + item = self.queue.get() + if item is None: + return # timeout + if isinstance(item, BaseException): + raise item + else: + yield item + + +class StubMessageWriter(MessageWriter): + """ + Replacement for the driver's MessageWriter, which it uses to send messages + to Felix. + + Buffers the messages and flush calls in a queue for the test to + interrogate. + """ + def __init__(self, sck): + super(StubMessageWriter, self).__init__(sck) + self.queue = Queue() + self.exception = None + + def send_message(self, msg_type, fields=None, flush=True): + if self.exception: + raise self.exception + self.queue.put((msg_type, fields)) + if flush: + self.flush() + + def next_msg(self): + return self.queue.get(timeout=1) + + def flush(self): + self.queue.put(FLUSH) + + +class PipeFile(object): + def __init__(self): + self.queue = Queue() + self.buf = None + self._finished = False + + def read(self, length): + data = "" + if not self.buf: + self.buf = self.queue.get() + while len(data) < length: + if isinstance(self.buf, BaseException): + raise self.buf + data += self.buf[:length - len(data)] + self.buf = self.buf[length - len(data):] + if not self.buf: + try: + self.buf = self.queue.get_nowait() + except Empty: + break + return data + + def write(self, data): + self.queue.put(data) + if data == "" or isinstance(data, Exception): + self._finished = True + + def __del__(self): + assert self._finished, "PipeFile wasn't correctly finished." + + +class StubRequest(object): + def __init__(self, stub_etcd, key, kwargs): + self.stub_etcd = stub_etcd + self.thread = threading.current_thread() + self.key = key + self.kwargs = kwargs + self.response = None + self.response_available = threading.Event() + self.pipe_file = None + + def __str__(self): + return "Request" % (self.key, + self.kwargs, + self.thread) + + def respond_with_exception(self, exc): + """ + Called from the test to raise an exception from the current/next + request. + """ + self.response = exc + self.on_response_avail() + + def on_response_avail(self): + self.response_available.set() + self.stub_etcd.on_req_closed(self) + + def respond_with_value(self, key, value, dir=False, mod_index=None, + etcd_index=None, status=200, action="get"): + """ + Called from the test to return a simple single-key value to the + driver. + """ + node = {"key": key, "value": value, "modifiedIndex": mod_index} + if dir: + node["dir"] = True + data = json.dumps({ + "action": action, + "node": node + }) + self.respond_with_data(data, etcd_index, status) + + def respond_with_dir(self, key, children, mod_index=None, + etcd_index=None, status=200): + """ + Called from the test to return a directory of key/values (from a + recursive request). + """ + nodes = [] + for k, v in children.iteritems(): + if v is not None: + nodes.append({"key": k, "value": v, + "modifiedIndex": mod_index}) + else: + nodes.append({"key": k, "dir": True, + "modifiedIndex": mod_index, + "nodes": []}) + data = json.dumps({ + "action": "get", + "node": { + "key": key, + "dir": True, + "modifiedIndex": mod_index, + "nodes": nodes + } + }) + self.respond_with_data(data, etcd_index, status) + + def respond_with_data(self, data, etcd_index, status): + """ + Called from the test to return a raw response (e.g. to send + malformed JSON). + """ + headers = self.stub_etcd.headers.copy() + if etcd_index is not None: + headers["x-etcd-index"] = str(etcd_index) + resp = MockResponse(status, data, headers) + self.response = resp + self.on_response_avail() + + def respond_with_stream(self, etcd_index, status=200): + """ + Called from the test to respond with a stream, allowing the test to + send chunks of data in response. + """ + headers = self.stub_etcd.headers.copy() + if etcd_index is not None: + headers["x-etcd-index"] = str(etcd_index) + self.pipe_file = PipeFile() + resp = MockResponse(status, self.pipe_file, headers) + self.response = resp + self.response_available.set() # We leave the req open in StubEtcd. + return self.pipe_file + + def get_response(self): + self.response_available.wait(timeout=30) # returns None in Python 2.6 + if self.response_available.is_set(): + return self.response + else: + raise AssertionError("No response") + + def assert_request(self, expected_key, **expected_args): + """ + Asserts the properies of the next request. + """ + default_args = {'wait_index': None, + 'preload_content': None, + 'recursive': False, + 'timeout': 5} + key = self.key + args = self.kwargs + for k, v in default_args.iteritems(): + if k in args and args[k] == v: + del args[k] + if expected_key != key: + raise AssertionError("Expected request for %s but got %s" % + (expected_key, key)) + if expected_args != args: + raise AssertionError("Expected request args %s for %s but got %s" % + (expected_args, key, args)) + + def stop(self): + if self.response_available.is_set(): + if self.pipe_file: + self.pipe_file.write(SystemExit()) + else: + self.respond_with_exception(SystemExit()) + + +class StubEtcd(object): + """ + A fake connection to etcd. We hook the driver's _issue_etcd_request + method and block the relevant thread until the test calls one of the + respond_... methods. + """ + def __init__(self): + self.request_queue = Queue() + self.response_queue = Queue() + self.headers = { + "x-etcd-cluster-id": "abcdefg" + } + self.lock = threading.Lock() + self.open_reqs = set() + + def request(self, key, **kwargs): + """ + Called from the driver to make a request. Blocks until the + test thread sends a response. + """ + _log.info("New request on thread %s: %s, %s", + threading.current_thread(), + key, kwargs) + request = StubRequest(self, key, kwargs) + with self.lock: + self.open_reqs.add(request) + rq = self.request_queue + if rq is None: + _log.warn("Request after shutdown: %s, %s", key, kwargs) + raise SystemExit() + else: + rq.put(request) + response = request.get_response() + if isinstance(response, BaseException): + raise response + else: + return response + + def get_next_request(self): + """ + Called from the test to get the next request from the driver. + """ + _log.info("Waiting for next request") + req = self.request_queue.get(timeout=1) + _log.info("Got request %s", req) + return req + + def assert_request(self, expected_key, **expected_args): + """ + Asserts the properies of the next request. + """ + req = self.request_queue.get(timeout=1) + req.assert_request(expected_key, **expected_args) + return req + + def on_req_closed(self, req): + with self.lock: + self.open_reqs.remove(req) + + def stop(self): + _log.info("Stopping stub etcd") + with self.lock: + _log.info("stop() got rq_lock") + while True: + try: + req = self.request_queue.get_nowait() + except Empty: + break + else: + self.open_reqs.add(req) + self.request_queue = None + for req in list(self.open_reqs): + _log.info("Aborting request %s", req) + req.stop() + _log.info("Stub etcd stopped; future requests should self-abort") + + +class MockResponse(object): + def __init__(self, status, data_or_exc, headers=None): + self.status = status + self._data_or_exc = data_or_exc + self.headers = headers or {} + + @property + def data(self): + if isinstance(self._data_or_exc, Exception): + raise self._data_or_exc + elif hasattr(self._data_or_exc, "read"): + return self._data_or_exc.read() + else: + return self._data_or_exc + + def read(self, *args): + return self._data_or_exc.read(*args) + + def getheader(self, header, default=None): + _log.debug("Asked for header %s", header) + return self.headers.get(header.lower(), default) \ No newline at end of file diff --git a/calico/etcddriver/test/test_driver.py b/calico/etcddriver/test/test_driver.py new file mode 100644 index 0000000000..337c9632fe --- /dev/null +++ b/calico/etcddriver/test/test_driver.py @@ -0,0 +1,929 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.etcddriver.test.test_driver +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Tests for the etcd driver module. +""" +import json +import threading +import traceback +from Queue import Empty + +from StringIO import StringIO +from unittest import TestCase + +import sys +from mock import Mock, patch, call +from urllib3 import HTTPConnectionPool +from urllib3.exceptions import TimeoutError, HTTPError +from calico.datamodel_v1 import READY_KEY, CONFIG_DIR, VERSION_DIR +from calico.etcddriver import driver +from calico.etcddriver.driver import ( + EtcdDriver, DriverShutdown, ResyncRequired, WatcherDied, ijson +) +from calico.etcddriver.protocol import * +from calico.etcddriver.test.stubs import ( + StubMessageReader, StubMessageWriter, StubEtcd, + FLUSH) + +_log = logging.getLogger(__name__) + + +patch.object = getattr(patch, "object") # Keep PyCharm linter happy. + + +class TestEtcdDriverFV(TestCase): + """ + FV-level tests for the driver. These tests run a real copy of the driver + but they stub out the felix socket and requests to etcd. + """ + + def setUp(self): + sck = Mock() + self.watcher_etcd = StubEtcd() + self.resync_etcd = StubEtcd() + + self.driver = EtcdDriver(sck) + self.msg_reader = StubMessageReader(sck) + self.msg_writer = StubMessageWriter(sck) + self.driver._msg_reader = self.msg_reader + self.driver._msg_writer = self.msg_writer + self.driver._issue_etcd_request = Mock( + spec=self.driver._issue_etcd_request, + side_effect=self.mock_etcd_request + ) + + self._logging_patch = patch("calico.etcddriver.driver." + "complete_logging", autospec=True) + self._logging_patch.start() + + def test_mainline_resync(self): + """ + Test of the mainline resync-and-merge processing. + + * Does the initial config handshake with Felix. + * Interleaves the snapshot response with updates via the watcher. + * Checks that the result is correctly merged. + """ + # Initial handshake. + self.start_driver_and_handshake() + # Check for etcd request and start the response. + snap_stream, watcher_req = self.start_snapshot_response() + # Respond to the watcher, this should get merged into the event + # stream at some point later. + watcher_req.respond_with_value( + "/calico/v1/adir/bkey", + "b", + mod_index=12, + action="set" + ) + # Wait until the watcher makes its next request (with revved + # wait_index) to make sure it has queued its event to the resync + # thread. + watcher_req = self.watcher_etcd.assert_request( + VERSION_DIR, recursive=True, timeout=90, wait_index=13 + ) + # Write some more data to the resync thread, it should process that + # and the queued watcher event. + snap_stream.write(''' + { + "key": "/calico/v1/adir/ckey", + "value": "c", + "modifiedIndex": 8 + }, + ''') + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/ckey", + MSG_KEY_VALUE: "c", + }) + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/bkey", + MSG_KEY_VALUE: "b", + }) + # Respond to the watcher with another event. + watcher_req.respond_with_value( + "/calico/v1/adir2/dkey", + "d", + mod_index=13, + action="set" + ) + # Wait until the watcher makes its next request (with revved + # wait_index) to make sure it has queued its event to the resync + # thread. + watcher_req = self.watcher_etcd.assert_request( + VERSION_DIR, recursive=True, timeout=90, wait_index=14 + ) + # Send the resync thread some data that should be ignored due to the + # preceding event. + snap_stream.write(''' + { + "key": "/calico/v1/adir/bkey", + "value": "b", + "modifiedIndex": 9 + }, + ''') + # The resync event would be generated first but we should should only + # see the watcher event. + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir2/dkey", + MSG_KEY_VALUE: "d", + }) + # Finish the snapshot. + snap_stream.write(''' + { + "key": "/calico/v1/Ready", + "value": "true", + "modifiedIndex": 10 + }] + }] + } + } + ''') + snap_stream.write("") # Close the response. + # Should get the in-sync message. (No event for Ready flag due to + # HWM. + self.assert_status_message(STATUS_IN_SYNC) + # Now send a watcher event, which should go straight through. + self.send_watcher_event_and_assert_felix_msg(14, req=watcher_req) + + # Check the contents of the trie. + keys = set(self.driver._hwms._hwms.keys()) + self.assertEqual(keys, set([u'/calico/v1/Ready/', + u'/calico/v1/adir/akey/', + u'/calico/v1/adir/bkey/', + u'/calico/v1/adir/ckey/', + u'/calico/v1/adir2/dkey/', + u'/calico/v1/adir/ekey/'])) + + def test_bad_data_triggers_resync(self): + # Initial handshake. + self.start_driver_and_handshake() + # Check for etcd request and start the response. + snap_stream, watcher_req = self.start_snapshot_response() + # Write some garbage to the stream, should trigger a resync. + watcher_stop_event = self.driver._watcher_stop_event + snap_stream.write(''' + { + "key + ''') + snap_stream.write("") + + watcher_stop_event.wait(1) + self.assertTrue(watcher_stop_event.is_set()) + self.assert_status_message(STATUS_WAIT_FOR_READY) + + def test_many_events_during_resync(self): + """ + Test many events during resync + + * Does the initial config handshake with Felix. + * Interleaves the snapshot response with updates via the watcher. + * Checks that the result is correctly merged. + """ + # Initial handshake. + self.start_driver_and_handshake() + + # Check for etcd request and start the response. + snap_stream, watcher_req = self.start_snapshot_response() + + # Respond to the watcher, this should get merged into the event + # stream at some point later. + for ii in xrange(200): + watcher_req.respond_with_value( + "/calico/v1/adir/bkey", + "watch", + mod_index=11 + ii, + action="set" + ) + watcher_req = self.watcher_etcd.assert_request( + VERSION_DIR, recursive=True, timeout=90, wait_index=12 + ii + ) + snap_stream.write(''' + { + "key": "/calico/v1/adir/bkey", + "value": "snap", + "modifiedIndex": 8 + }, + { + "key": "/calico/v1/Ready", + "value": "true", + "modifiedIndex": 10 + }] + }] + } + } + ''') + snap_stream.write("") + + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/bkey", + MSG_KEY_VALUE: "snap", + }) + for _ in xrange(200): + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/bkey", + MSG_KEY_VALUE: "watch", + }) + self.assert_status_message(STATUS_IN_SYNC) + + def test_felix_triggers_resync(self): + self._run_initial_resync() + + # Wait for the watcher to make its request. + watcher_req = self.watcher_etcd.assert_request( + VERSION_DIR, recursive=True, timeout=90, wait_index=15 + ) + + # Send a resync request from Felix. + self.msg_reader.send_msg(MSG_TYPE_RESYNC, {}) + + # Respond to the watcher, this should trigger the resync. + watcher_req.respond_with_value( + "/calico/v1/adir/ekey", + "e", + mod_index=15, + action="set" + ) + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/ekey", + MSG_KEY_VALUE: "e", + }) + self.assert_flush_to_felix() + + self.assert_status_message(STATUS_WAIT_FOR_READY) + + # Re-do the config handshake. + self.do_handshake() + + # We should get a request to load the full snapshot. + watcher_req = self.resync_etcd.assert_request( + VERSION_DIR, recursive=True, timeout=120, preload_content=False + ) + snap_stream = watcher_req.respond_with_stream( + etcd_index=100 + ) + + watcher_req = self.watcher_etcd.assert_request(VERSION_DIR, + wait_index=16, + recursive=True, + timeout=90) + watcher_req.respond_with_value("/calico/v1/adir/ekey", "e", + mod_index=50, action="set") + + # Wait for next watcher event to make sure it has queued its request to + # the resync thread. + watcher_req = self.watcher_etcd.assert_request(VERSION_DIR, + wait_index=51, + recursive=True, + timeout=90) + + # Start sending the snapshot response: + snap_stream.write('''{ + "action": "get", + "node": { + "key": "/calico/v1", + "dir": true, + "nodes": [ + { + "key": "/calico/v1/adir", + "dir": true, + "nodes": [ + { + "key": "/calico/v1/adir/akey", + "value": "akey's value", + "modifiedIndex": 98 + }, + ''') + # Should generate a message to felix even though it's only seen part + # of the response... + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/akey", + MSG_KEY_VALUE: "akey's value", + }) + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/ekey", + MSG_KEY_VALUE: "e", + }) + + # Respond to the watcher, this should get merged into the event + # stream at some point later. + watcher_req.respond_with_value( + "/calico/v1/adir/bkey", + "b", + mod_index=102, + action="set" + ) + + # Wait until the watcher makes its next request (with revved + # wait_index) to make sure it has queued its event to the resync + # thread. Skip any events fro the old watcher. + watcher_req = self.watcher_etcd.assert_request(VERSION_DIR, + wait_index=103, + recursive=True, + timeout=90) + + # Write some data for an unchanged key to the resync thread, which + # should be ignored. + snap_stream.write(''' + { + "key": "/calico/v1/adir/ckey", + "value": "c", + "modifiedIndex": 8 + }, + ''') + # But we should get the watcher update. + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/bkey", + MSG_KEY_VALUE: "b", + }) + # Finish the snapshot. + snap_stream.write(''' + { + "key": "/calico/v1/adir2/dkey", + "value": "c", + "modifiedIndex": 8 + }, + { + "key": "/calico/v1/Ready", + "value": "true", + "modifiedIndex": 10 + }] + }] + } + } + ''') + snap_stream.write("") # Close the response. + # Should get a deletion for the keys that were missing in this + # snapshot. + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/ekey", + MSG_KEY_VALUE: None, + }) + # Should get the in-sync message. (No event for Ready flag due to + # HWM. + self.assert_status_message(STATUS_IN_SYNC) + # Now send a watcher event, which should go straight through. + self.send_watcher_event_and_assert_felix_msg(104, req=watcher_req) + + def test_directory_deletion(self): + self._run_initial_resync() + # For coverage: Nothing happens for a while, poll times out. + watcher_req = self.watcher_etcd.get_next_request() + watcher_req.respond_with_exception( + driver.ReadTimeoutError(Mock(), "", "") + ) + # For coverage: Then a set to a dir, which should be ignored. + watcher_req = self.watcher_etcd.get_next_request() + watcher_req.respond_with_data(json.dumps({ + "action": "create", + "node": { + "key": "/calico/v1/foo", + "dir": True + } + }), 100, 200) + # Then a whole directory is deleted. + watcher_req = self.watcher_etcd.get_next_request() + watcher_req.respond_with_value( + "/calico/v1/adir", + dir=True, + value=None, + action="delete", + mod_index=101, + status=300 # For coverage of warning log. + ) + # Should get individual deletes for each one then a flush. We're + # relying on the trie returning sorted results here. + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/akey", + MSG_KEY_VALUE: None, + }) + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/bkey", + MSG_KEY_VALUE: None, + }) + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/ckey", + MSG_KEY_VALUE: None, + }) + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/ekey", + MSG_KEY_VALUE: None, + }) + self.assert_flush_to_felix() + + # Check the contents of the trie. + keys = set(self.driver._hwms._hwms.keys()) + self.assertEqual(keys, set([u'/calico/v1/Ready/', + u'/calico/v1/adir2/dkey/'])) + + def _run_initial_resync(self): + try: + # Start by going through the first resync. + self.test_mainline_resync() # Returns open watcher req. + except AssertionError: + _log.exception("Mainline resync test failed, aborting test %s", + self.id()) + raise AssertionError("Mainline resync test failed to " + "initialise driver") + + def test_root_directory_deletion(self): + self._run_initial_resync() + # Delete the whole /calico/v1 dir. + watcher_req = self.watcher_etcd.get_next_request() + watcher_req.respond_with_data(json.dumps({ + "action": "delete", + "node": { + "key": "/calico/v1/", + "dir": True + } + }), 100, 200) + + # Should trigger a resync. + self.assert_status_message(STATUS_WAIT_FOR_READY) + + def test_garbage_watcher_response(self): + self._run_initial_resync() + # Delete the whole /calico/v1 dir. + watcher_req = self.watcher_etcd.get_next_request() + watcher_req.respond_with_data("{foobar", 100, 200) + + # Should trigger a resync. + self.assert_status_message(STATUS_WAIT_FOR_READY) + + def send_watcher_event_and_assert_felix_msg(self, etcd_index, req=None): + if req is None: + req = self.watcher_etcd.get_next_request() + req.respond_with_value( + "/calico/v1/adir/ekey", + "e", + mod_index=etcd_index, + action="set" + ) + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/ekey", + MSG_KEY_VALUE: "e", + }) + self.assert_flush_to_felix() + + @patch("time.sleep", autospec=True) + def test_resync_pipe_write_fail(self, m_sleep): + """ + Test a read failure on the snapshot. + """ + # Start the driver, it will wait for a message from Felix. + self.driver.start() + # Queue up an error on the driver's next write. + self.msg_writer.exception = WriteFailed() + # Send init message from Felix to driver. + self.send_init_msg() + # Driver should die. + for _ in xrange(100): + # Need to time out the reader thread or it will block shutdown. + self.msg_reader.send_timeout() + if self.driver.join(timeout=0.01): + break + else: + self.fail("Driver failed to die.") + + @patch("time.sleep", autospec=True) + def test_resync_etcd_read_fail(self, m_sleep): + """ + Test a read failure on the snapshot. + """ + # Initial handshake. + self.start_driver_and_handshake() + # Start streaming some data. + snap_stream, watcher_req = self.start_snapshot_response() + # But then the read times out... + snap_stream.write(TimeoutError()) + # Triggering a restart of the resync loop. + self.assert_status_message(STATUS_WAIT_FOR_READY) + + @patch("time.sleep") + def test_bad_ready_key_retry(self, m_sleep): + self.start_driver_and_init() + # Respond to etcd request with a bad response + req = self.resync_etcd.assert_request(READY_KEY) + req.respond_with_data("foobar", 123, 500) + # Then it should retry. + self.resync_etcd.assert_request(READY_KEY) + m_sleep.assert_called_once_with(1) + + def start_driver_and_init(self): + self.driver.start() + # First message comes from Felix. + self.send_init_msg() + # Should trigger driver to send a status and start polling the ready + # flag. + self.assert_status_message(STATUS_WAIT_FOR_READY) + + def start_driver_and_handshake(self): + self.start_driver_and_init() + self.do_handshake() + + def do_handshake(self): + # Respond to etcd request with ready == true. + req = self.resync_etcd.assert_request(READY_KEY) + req.respond_with_value(READY_KEY, "true", mod_index=10) + # Then etcd should get the global config request. + req = self.resync_etcd.assert_request(CONFIG_DIR, recursive=True) + req.respond_with_dir(CONFIG_DIR, { + CONFIG_DIR + "/InterfacePrefix": "tap", + CONFIG_DIR + "/Foo": None, # Directory + }) + # Followed by the per-host one... + req = self.resync_etcd.assert_request( + "/calico/v1/host/thehostname/config", recursive=True + ) + req.respond_with_data('{"errorCode": 100}', + 10, 404) + # Then the driver should send the config to Felix. + self.assert_msg_to_felix( + MSG_TYPE_CONFIG_LOADED, + { + MSG_KEY_GLOBAL_CONFIG: {"InterfacePrefix": "tap"}, + MSG_KEY_HOST_CONFIG: {}, + } + ) + self.assert_flush_to_felix() + # We respond with the config message to trigger the start of the + # resync. + self.msg_reader.send_msg( + MSG_TYPE_CONFIG, + { + MSG_KEY_LOG_FILE: "/tmp/driver.log", + MSG_KEY_SEV_FILE: "DEBUG", + MSG_KEY_SEV_SCREEN: "DEBUG", + MSG_KEY_SEV_SYSLOG: "DEBUG", + } + ) + self.assert_status_message(STATUS_RESYNC) + + def start_snapshot_response(self, etcd_index=10): + # We should get a request to load the full snapshot. + req = self.resync_etcd.assert_request( + VERSION_DIR, recursive=True, timeout=120, preload_content=False + ) + snap_stream = req.respond_with_stream( + etcd_index=etcd_index + ) + # And then the headers should trigger a request from the watcher + # including the etcd_index we sent even though we haven't sent a + # response body to the resync thread. + req = self.watcher_etcd.assert_request( + VERSION_DIR, recursive=True, timeout=90, wait_index=etcd_index+1 + ) + # Start sending the snapshot response: + snap_stream.write('''{ + "action": "get", + "node": { + "key": "/calico/v1", + "dir": true, + "nodes": [ + { + "key": "/calico/v1/adir", + "dir": true, + "nodes": [ + { + "key": "/calico/v1/adir/akey", + "value": "akey's value", + "modifiedIndex": %d + }, + ''' % (etcd_index - 2)) + # Should generate a message to felix even though it's only seen part + # of the response... + self.assert_msg_to_felix(MSG_TYPE_UPDATE, { + MSG_KEY_KEY: "/calico/v1/adir/akey", + MSG_KEY_VALUE: "akey's value", + }) + return snap_stream, req + + def assert_status_message(self, status): + _log.info("Expecting %s status from driver...", status) + self.assert_msg_to_felix( + MSG_TYPE_STATUS, + {MSG_KEY_STATUS: status} + ) + self.assert_flush_to_felix() + + def send_init_msg(self): + self.msg_reader.send_msg( + MSG_TYPE_INIT, + { + MSG_KEY_ETCD_URL: "http://localhost:4001", + MSG_KEY_HOSTNAME: "thehostname", + } + ) + + def assert_msg_to_felix(self, msg_type, fields=None): + try: + mt, fs = self.msg_writer.next_msg() + except Empty: + self.fail("Expected %s message to felix but no message was sent" % + msg_type) + self.assertEqual(msg_type, mt, msg="Unexpected message: %s" % fs) + self.assertEqual(fields, fs, msg="Unexpected message: %s" % fs) + + def assert_flush_to_felix(self): + self.assertEqual(self.msg_writer.queue.get(timeout=10), + FLUSH) + + def assert_no_msgs(self): + try: + msg = self.msg_writer.queue.get(timeout=1) + except Empty: + pass + else: + self.fail("Message unexpectedly received: %s" % msg) + + def mock_etcd_request(self, http_pool, key, timeout=5, wait_index=None, + recursive=False, preload_content=None): + """ + Called from another thread when the driver makes an etcd request, + we queue the request via the correct stub, then block, waiting + for the main thread to tell us what to do. + """ + if http_pool is self.driver._resync_http_pool: + _log.info("Resync thread issuing request for %s timeout=%s, " + "wait_index=%s, recursive=%s, preload=%s", key, timeout, + wait_index, recursive, preload_content) + etcd_stub = self.resync_etcd + else: + _log.info("Watcher thread issuing request for %s timeout=%s, " + "wait_index=%s, recursive=%s, preload=%s", key, timeout, + wait_index, recursive, preload_content) + etcd_stub = self.watcher_etcd + + return etcd_stub.request(key, + timeout=timeout, + wait_index=wait_index, + recursive=recursive, + preload_content=preload_content) + + def tearDown(self): + _log.info("Tearing down test") + try: + # Request that the driver stops. + self.driver.stop() + # Make sure we don't block the driver from stopping. + self.msg_reader.send_timeout() + + # SystemExit kills (only) the thread silently. + self.resync_etcd.stop() + self.watcher_etcd.stop() + # Wait for it to stop. + if not self.driver.join(1): + dump_all_thread_stacks() + self.fail("Driver failed to stop") + finally: + # Now the driver is stopped, it's safe to remove our patch of + # complete_logging() + self._logging_patch.stop() + + +class TestDriver(TestCase): + """ + Unit-test tests of the Driver. + """ + def setUp(self): + self.m_sck = Mock(spec=socket.socket) + self.driver = EtcdDriver(self.m_sck) + self.msg_reader = StubMessageReader(self.m_sck) + self.msg_writer = StubMessageWriter(self.m_sck) + self.driver._msg_reader = self.msg_reader + self.driver._msg_writer = self.msg_writer + + def test_read_bad_message(self): + self.msg_reader.send_msg("unknown", {}) + self.assertRaises(RuntimeError, self.driver._read_from_socket) + + def test_shutdown_before_config(self): + self.driver._stop_event.set() + self.assertRaises(DriverShutdown, self.driver._wait_for_config) + + def test_shutdown_before_ready(self): + self.driver._stop_event.set() + self.assertRaises(DriverShutdown, self.driver._wait_for_ready) + + def test_issue_etcd_request_basic_get(self): + # Initialise the etcd URL. + self.driver._handle_init({ + MSG_KEY_ETCD_URL: "http://localhost:4001/", + MSG_KEY_HOSTNAME: "ourhost", + }) + m_pool = Mock(spec=HTTPConnectionPool) + self.driver._issue_etcd_request(m_pool, "calico/v1/Ready") + self.assertEqual( + m_pool.request.mock_calls, + [call("GET", + "http://localhost:4001/v2/keys/calico/v1/Ready", + fields=None, + timeout=5, + preload_content=True)] + ) + + def test_issue_etcd_request_recursive_watch(self): + # Initialise the etcd URL. + self.driver._handle_init({ + MSG_KEY_ETCD_URL: "http://localhost:4001/", + MSG_KEY_HOSTNAME: "ourhost", + }) + m_pool = Mock(spec=HTTPConnectionPool) + self.driver._issue_etcd_request(m_pool, "calico/v1", timeout=10, + wait_index=11, recursive=True) + self.assertEqual( + m_pool.request.mock_calls, + [call("GET", + "http://localhost:4001/v2/keys/calico/v1", + fields={"recursive": "true", + "wait": "true", + "waitIndex": 11}, + timeout=10, + preload_content=False)] + ) + + def test_cluster_id_check(self): + m_resp = Mock() + m_resp.getheader.return_value = "abcdef" + self.driver._check_cluster_id(m_resp) + m_resp = Mock() + m_resp.getheader.return_value = "ghijkl" + self.assertRaises(DriverShutdown, self.driver._check_cluster_id, + m_resp) + self.assertTrue(self.driver._stop_event.is_set()) + + def test_load_config_bad_data(self): + with patch.object(self.driver, "_etcd_request") as m_etcd_req: + m_resp = Mock() + m_resp.data = "{garbage" + m_etcd_req.return_value = m_resp + self.assertRaises(ResyncRequired, + self.driver._load_config, "/calico/v1/config") + + def test_start_snap_missing_cluster_id(self): + with patch.object(self.driver, "_etcd_request") as m_etcd_req: + m_resp = Mock() + m_resp.getheader.return_value = 123 + m_etcd_req.return_value = m_resp + self.assertRaises(ResyncRequired, + self.driver._start_snapshot_request) + + def test_cluster_id_missing(self): + m_resp = Mock() + m_resp.getheader.return_value = None + self.driver._check_cluster_id(m_resp) + self.assertEqual(m_resp.getheader.mock_calls, + [call("x-etcd-cluster-id")]) + + def test_watcher_dies_during_resync(self): + self.driver.stop() + with patch.object(self.driver, "_on_key_updated") as m_on_key: + with patch.object(self.driver, + "_handle_next_watcher_event") as m_handle: + m_queue = Mock() + m_queue.empty.return_value = False + m_handle.side_effect = WatcherDied() + self.driver._watcher_queue = m_queue + self.assertRaises(DriverShutdown, + self.driver._handle_etcd_node, + 123, "/calico/v1/foo", "bar", + snapshot_index=1000) + + def test_handle_next_watcher_died(self): + self.driver._watcher_queue = None + self.assertRaises(WatcherDied, self.driver._handle_next_watcher_event, + False) + + def test_handle_next_queue_empty(self): + m_queue = Mock() + m_queue.get.side_effect = iter([ + Empty(), + RuntimeError() + ]) + self.driver._watcher_queue = m_queue + self.assertRaises(RuntimeError, + self.driver._handle_next_watcher_event, + False) + + def test_handle_next_stopped(self): + self.driver._watcher_queue = Mock() + self.driver.stop() + self.assertRaises(DriverShutdown, + self.driver._handle_next_watcher_event, + False) + + def test_ready_key_set_to_false(self): + self.assertRaises(ResyncRequired, + self.driver._on_key_updated, READY_KEY, "false") + + def test_watch_etcd_error_from_etcd(self): + m_queue = Mock() + m_stop_ev = Mock() + m_stop_ev.is_set.return_value = False + with patch.object(self.driver, "get_etcd_connection") as m_get_conn: + with patch.object(self.driver, "_etcd_request") as m_req: + with patch.object(self.driver, "_check_cluster_id") as m_check: + m_resp = Mock() + m_resp.data = json.dumps({"errorCode": 100}) + m_req.side_effect = iter([ + m_resp, + AssertionError() + ]) + self.driver.watch_etcd(10, m_queue, m_stop_ev) + + def test_parse_snapshot_bad_status(self): + m_resp = Mock() + m_resp.status = 500 + self.assertRaises(ResyncRequired, driver.parse_snapshot, + m_resp, Mock()) + + def test_parse_snapshot_bad_data(self): + m_resp = Mock() + m_resp.status = 200 + m_resp.read.return_value = "[]" + self.assertRaises(ResyncRequired, driver.parse_snapshot, + m_resp, Mock()) + + def test_parse_snapshot_garbage_data(self): + m_resp = Mock() + m_resp.status = 200 + m_resp.read.return_value = "garbage" + self.assertRaises(ResyncRequired, driver.parse_snapshot, + m_resp, Mock()) + + def test_resync_driver_stopped(self): + self.driver._init_received.set() + with patch.object(self.driver, "get_etcd_connection") as m_get: + m_get.side_effect = DriverShutdown() + self.driver._resync_and_merge() + + @patch("time.sleep") + def test_resync_http_error(self, m_sleep): + self.driver._init_received.set() + with patch.object(self.driver, "get_etcd_connection") as m_get: + with patch("calico.etcddriver.driver.monotonic_time") as m_time: + m_time.side_effect = iter([ + 1, 10, RuntimeError() + ]) + m_get.side_effect = HTTPError() + self.assertRaises(RuntimeError, self.driver._resync_and_merge) + + def test_parse_snap_error_from_etcd(self): + parser = ijson.parse(StringIO(json.dumps({ + "errorCode": 100 + }))) + next(parser) + self.assertRaises(ResyncRequired, driver._parse_map, parser, None) + + def test_parse_snap_bad_data(self): + parser = ijson.parse(StringIO(json.dumps({ + "nodes": [ + "foo" + ] + }))) + next(parser) + self.assertRaises(ValueError, driver._parse_map, parser, None) + + def test_join_not_stopped(self): + with patch.object(self.driver._stop_event, "wait"): + self.assertFalse(self.driver.join()) + + def test_process_events_stopped(self): + self.driver._stop_event.set() + self.assertRaises(DriverShutdown, self.driver._process_events_only) + + def test_watch_etcd_already_stopped(self): + stop_event = threading.Event() + stop_event.set() + m_queue = Mock() + self.driver.watch_etcd(10, m_queue, stop_event) + self.assertEqual(m_queue.put.mock_calls, [call(None)]) + + +def dump_all_thread_stacks(): + print >> sys.stderr, "\n*** STACKTRACE - START ***\n" + code = [] + for threadId, stack in sys._current_frames().items(): + code.append("\n# ThreadID: %s" % threadId) + for filename, lineno, name, line in traceback.extract_stack(stack): + code.append('File: "%s", line %d, in %s' % (filename, + lineno, name)) + if line: + code.append(" %s" % (line.strip())) + for line in code: + print >> sys.stderr, line + print >> sys.stderr, "\n*** STACKTRACE - END ***\n" \ No newline at end of file diff --git a/calico/etcddriver/test/test_hwm.py b/calico/etcddriver/test/test_hwm.py new file mode 100644 index 0000000000..a0dbca98c2 --- /dev/null +++ b/calico/etcddriver/test/test_hwm.py @@ -0,0 +1,153 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +test_hwm +~~~~~~~~ + +Tests for high water mark tracking function. +""" + +import logging +from unittest import TestCase +from mock import Mock, call, patch +from calico.etcddriver import hwm +from calico.etcddriver.hwm import HighWaterTracker + +_log = logging.getLogger(__name__) + + +class TestHighWaterTracker(TestCase): + def setUp(self): + self.hwm = HighWaterTracker() + + def test_mainline(self): + # Test merging of updates between a snapshot with etcd_index 10 and + # updates coming in afterwards with indexes 11, 12, ... + + # We use prefix "/a/$" because $ is not allowed in the trie so it + # implicitly tests encoding/decoding is being properly applied. + + old_hwm = self.hwm.update_hwm("/a/$/c", 9) # Pre-snapshot + self.assertEqual(old_hwm, None) + old_hwm = self.hwm.update_hwm("/b/c/d", 9) # Pre-snapshot + self.assertEqual(old_hwm, None) + old_hwm = self.hwm.update_hwm("/j/c/d", 9) # Pre-snapshot + self.assertEqual(old_hwm, None) + self.assertEqual(len(self.hwm), 3) + + # While merging a snapshot we track deletions. + self.hwm.start_tracking_deletions() + + # Send in some keys from the snapshot. + old_hwm = self.hwm.update_hwm("/a/$/c", 10) # From snapshot + self.assertEqual(old_hwm, 9) + old_hwm = self.hwm.update_hwm("/a/$/d", 10) # From snapshot + self.assertEqual(old_hwm, None) + old_hwm = self.hwm.update_hwm("/d/e/f", 10) # From snapshot + self.assertEqual(old_hwm, None) + self.assertEqual(len(self.hwm), 5) + + # This key is first seen in the event stream, so the snapshot version + # should be ignored. + old_hwm = self.hwm.update_hwm("/a/h/i", 11) # From events + self.assertEqual(old_hwm, None) + old_hwm = self.hwm.update_hwm("/a/h/i", 10) # From snapshot + self.assertEqual(old_hwm, 11) + old_hwm = self.hwm.update_hwm("/a/h/i", 12) # From events + self.assertEqual(old_hwm, 11) # Still 11, snapshot ignored. + self.assertEqual(len(self.hwm), 6) + + # Then a whole subtree gets deleted by the events. + deleted_keys = self.hwm.store_deletion("/a/$", 13) + self.assertEqual(set(deleted_keys), set(["/a/$/c", "/a/$/d"])) + self.assertEqual(len(self.hwm), 4) + + # But afterwards, we see a snapshot key within the subtree, it should + # be ignored. + old_hwm = self.hwm.update_hwm("/a/$/e", 10) + self.assertEqual(old_hwm, 13) # Returns the etcd_index of the delete. + # Then a new update from the event stream, recreates the directory. + old_hwm = self.hwm.update_hwm("/a/$/f", 14) + self.assertEqual(old_hwm, None) + self.assertEqual(len(self.hwm), 5) + # And subsequent updates are processed ignoring the delete. + old_hwm = self.hwm.update_hwm("/a/$/f", 15) + self.assertEqual(old_hwm, 14) + # However, snapshot updates from within the deleted subtree are still + # ignored. + old_hwm = self.hwm.update_hwm("/a/$/e", 10) + self.assertEqual(old_hwm, 13) # Returns the etcd_index of the delete. + old_hwm = self.hwm.update_hwm("/a/$/f", 10) + self.assertEqual(old_hwm, 13) # Returns the etcd_index of the delete. + old_hwm = self.hwm.update_hwm("/a/$/g", 10) + self.assertEqual(old_hwm, 13) # Returns the etcd_index of the delete. + self.assertEqual(len(self.hwm), 5) + # But ones outside the subtree ar not. + old_hwm = self.hwm.update_hwm("/f/g", 10) + self.assertEqual(old_hwm, None) + # And subsequent updates are processed ignoring the delete. + old_hwm = self.hwm.update_hwm("/a/$/f", 16) + self.assertEqual(old_hwm, 15) + + # End of snapshot: we stop tracking deletions, which should free up the + # resources. + self.hwm.stop_tracking_deletions() + self.assertEqual(self.hwm._deletion_hwms, None) + + # Then, subseqent updates should be handled normally. + old_hwm = self.hwm.update_hwm("/a/$/f", 17) + self.assertEqual(old_hwm, 16) # From previous event + old_hwm = self.hwm.update_hwm("/g/b/f", 18) + self.assertEqual(old_hwm, None) # Seen for the first time. + old_hwm = self.hwm.update_hwm("/d/e/f", 19) + self.assertEqual(old_hwm, 10) # From the snapshot. + self.assertEqual(len(self.hwm), 7) + + # We should be able to find all the keys that weren't seen during + # the snapshot. + old_keys = self.hwm.remove_old_keys(10) + self.assertEqual(set(old_keys), set(["/b/c/d", "/j/c/d"])) + self.assertEqual(len(self.hwm), 5) + + # They should now be gone from the index. + old_hwm = self.hwm.update_hwm("/b/c/d", 20) + self.assertEqual(old_hwm, None) + self.assertEqual(len(self.hwm), 6) + + +class TestKeyEncoding(TestCase): + def test_encode_key(self): + self.assert_enc_dec("/calico/v1/foo/bar", "/calico/v1/foo/bar/") + + self.assert_enc_dec("/:_-./foo", "/:_-./foo/") + self.assert_enc_dec("/:_-.~/foo", "/:_-.%7E/foo/") + self.assert_enc_dec("/%/foo", "/%25/foo/") + self.assert_enc_dec(u"/\u01b1/foo", "/%C6%B1/foo/") + self.assertEqual(hwm.encode_key("/foo/"), "/foo/") + + def assert_enc_dec(self, key, expected_encoding): + encoded = hwm.encode_key(key) + self.assertEqual( + encoded, + expected_encoding, + msg="Expected %r to encode as %r but got %r" % + (key, expected_encoding, encoded)) + decoded = hwm.decode_key(encoded) + self.assertEqual( + decoded, + key, + msg="Expected %r to decode as %r but got %r" % + (encoded, key, decoded)) + diff --git a/calico/etcddriver/test/test_main.py b/calico/etcddriver/test/test_main.py new file mode 100644 index 0000000000..e25d8b7b19 --- /dev/null +++ b/calico/etcddriver/test/test_main.py @@ -0,0 +1,89 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.etcddriver.test.test_main +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Test __main__ module. +""" + +import logging +from unittest import TestCase + +import sys +from mock import Mock, call, patch + +_log = logging.getLogger(__name__) + + +class TestMain(TestCase): + def setUp(self): + assert "calico.etcddriver.__main__" not in sys.modules + + @patch("os.getppid", autospec=True) + @patch("socket.socket", autospec=True) + @patch("calico.common.default_logging", autospec=True) + @patch("calico.etcddriver.driver.EtcdDriver", autospec=True) + def test_mainline(self, m_driver_cls, m_logging, m_socket, m_ppid): + m_ppid.return_value = 123 + m_driver = m_driver_cls.return_value + m_driver.join.side_effect = iter([ + False, + True + ]) + self._import_main() + self.assertEqual(m_driver.mock_calls, + [call.start(), + call.join(timeout=1), + call.join(timeout=1)]) + self.assertEqual(m_logging.mock_calls, + [call(gevent_in_use=False, + syslog_executable_name="calico-felix-etcd")]) + + @patch("os.getppid", autospec=True) + @patch("socket.socket", autospec=True) + @patch("calico.common.default_logging", autospec=True) + @patch("calico.etcddriver.driver.EtcdDriver", autospec=True) + def test_reparent(self, m_driver_cls, m_logging, m_socket, m_ppid): + m_ppid.side_effect = iter([123, 123, 1]) + m_driver = m_driver_cls.return_value + m_driver.join.return_value = False + self._import_main() + self.assertEqual(m_driver.mock_calls, + [call.start(), + call.join(timeout=1), + call.join(timeout=1), + call.stop()]) + + @patch("os.getppid", autospec=True) + @patch("socket.socket", autospec=True) + @patch("calico.common.default_logging", autospec=True) + @patch("calico.etcddriver.driver.EtcdDriver", autospec=True) + def test_connection_failure(self, m_driver_cls, m_logging, m_socket, + m_ppid): + m_ppid.side_effect = iter([123, 123, 1]) + m_sck = m_socket.return_value + m_sck.connect.side_effect = RuntimeError() + self.assertRaises(RuntimeError, self._import_main) + + def _import_main(self): + import calico.etcddriver.__main__ as main + _ = main # Keep linter happy + + def tearDown(self): + try: + del sys.modules["calico.etcddriver.__main__"] + except KeyError: + pass diff --git a/calico/etcddriver/test/test_protocol.py b/calico/etcddriver/test/test_protocol.py new file mode 100644 index 0000000000..14d36bf5ad --- /dev/null +++ b/calico/etcddriver/test/test_protocol.py @@ -0,0 +1,234 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.etcddriver.test_protocol +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Tests for Felix/etcddriver protocol read/write function. +""" + +import logging +import socket +from unittest import TestCase +import errno +from mock import Mock, call, patch +import msgpack +from calico.etcddriver.protocol import ( + MessageWriter, STATUS_RESYNC, MSG_KEY_STATUS, MSG_TYPE_STATUS, + MSG_KEY_TYPE, STATUS_IN_SYNC, MessageReader, + SocketClosed, WriteFailed) + +_log = logging.getLogger(__name__) + + +class StubWriterSocket(object): + def __init__(self): + self.chunks = [] + self.unpacker = msgpack.Unpacker() + self.exception = None + + def sendall(self, data): + if self.exception: + raise self.exception + self.chunks.append(data) + self.unpacker.feed(data) + + def next_msg(self): + return next(self.unpacker) + + +class TestMessageWriter(TestCase): + def setUp(self): + self.sck = StubWriterSocket() + self.writer = MessageWriter(self.sck) + self.unpacker = msgpack.Unpacker() + + def test_send_message(self): + self.writer.send_message(MSG_TYPE_STATUS, + { + MSG_KEY_STATUS: STATUS_RESYNC + }) + self.assert_message_sent({ + MSG_KEY_TYPE: MSG_TYPE_STATUS, + MSG_KEY_STATUS: STATUS_RESYNC + }) + self.assert_no_more_messages() + + def test_send_message_error(self): + self.sck.exception = socket.error() + self.assertRaises(WriteFailed, self.writer.send_message, + MSG_TYPE_STATUS, + { + MSG_KEY_STATUS: STATUS_RESYNC + }) + + def test_send_message_buffered(self): + # First message gets buffered. + self.writer.send_message(MSG_TYPE_STATUS, + flush=False) + self.assert_no_more_messages() + + # Second message triggers a flush of both messages, in order. + self.writer.send_message(MSG_TYPE_STATUS, + { + MSG_KEY_STATUS: STATUS_IN_SYNC + }) + self.assert_message_sent({ + MSG_KEY_TYPE: MSG_TYPE_STATUS + }) + self.assert_message_sent({ + MSG_KEY_TYPE: MSG_TYPE_STATUS, + MSG_KEY_STATUS: STATUS_IN_SYNC + }) + self.assert_no_more_messages() + + def test_eventual_flush(self): + # First 200 messages should be buffered. + for _ in xrange(200): + self.writer.send_message(MSG_TYPE_STATUS, + { + MSG_KEY_STATUS: STATUS_RESYNC + }, + flush=False) + self.assert_no_more_messages() + + # 201st message triggers them all to be sent. + self.writer.send_message(MSG_TYPE_STATUS, + { + MSG_KEY_STATUS: STATUS_RESYNC + }, + flush=False) + for _ in xrange(201): + self.assert_message_sent({ + MSG_KEY_TYPE: MSG_TYPE_STATUS, + MSG_KEY_STATUS: STATUS_RESYNC + }) + self.assert_no_more_messages() + + def test_flush_no_content(self): + self.writer.flush() + self.assertFalse(self.sck.chunks) + + def assert_message_sent(self, msg): + try: + received_msg = self.sck.next_msg() + except StopIteration: + self.fail("No messages received") + self.assertEqual(received_msg, msg, + "Received incorrect message: %s " + "while expecting: %s" % (received_msg, msg)) + + def assert_no_more_messages(self): + try: + msg = self.sck.next_msg() + except StopIteration: + return + else: + self.fail("Unexpected message: %s" % msg) + + +class TestMessageReader(TestCase): + def setUp(self): + self.sck = Mock(spec=socket.socket) + self.reader = MessageReader(self.sck) + + @patch("select.select", autospec=True) + def test_mainline(self, m_select): + m_select.side_effect = iter([ + ([self.sck], [], []), + ([self.sck], [], []), + ]) + exp_msg = {MSG_KEY_TYPE: MSG_TYPE_STATUS, + MSG_KEY_STATUS: STATUS_RESYNC} + self.sck.recv.return_value = msgpack.dumps(exp_msg) + for _ in xrange(2): + msg_gen = self.reader.new_messages(timeout=1) + msg_type, msg = next(msg_gen) + self.assertEqual(msg_type, MSG_TYPE_STATUS) + self.assertEqual(msg, exp_msg) + self.assertEqual( + self.sck.recv.mock_calls, + [ + call(16384), + call(16384), + ] + ) + + @patch("select.select", autospec=True) + def test_partial_read(self, m_select): + m_select.side_effect = iter([ + ([self.sck], [], []), + ([self.sck], [], []), + ]) + exp_msg = {MSG_KEY_TYPE: MSG_TYPE_STATUS} + msg_bytes = msgpack.dumps(exp_msg) + self.sck.recv.side_effect = iter([ + msg_bytes[:len(msg_bytes)/2], + msg_bytes[len(msg_bytes)/2:], + ]) + self.assertRaises(StopIteration, next, + self.reader.new_messages(timeout=None)) + self.assertEqual(next(self.reader.new_messages(timeout=None)), + (MSG_TYPE_STATUS, exp_msg)) + + @patch("select.select", autospec=True) + def test_retryable_error(self, m_select): + m_select.side_effect = iter([ + ([self.sck], [], []), + ([self.sck], [], []), + ([self.sck], [], []), + ([self.sck], [], []), + ]) + errors = [] + for no in [errno.EAGAIN, errno.EWOULDBLOCK, errno.EINTR]: + err = socket.error() + err.errno = no + errors.append(err) + exp_msg = {MSG_KEY_TYPE: MSG_TYPE_STATUS, + MSG_KEY_STATUS: STATUS_RESYNC} + self.sck.recv.side_effect = iter(errors + [msgpack.dumps(exp_msg)]) + for _ in errors: + msg_gen = self.reader.new_messages(timeout=1) + self.assertRaises(StopIteration, next, msg_gen) + msg_gen = self.reader.new_messages(timeout=1) + msg_type, msg = next(msg_gen) + self.assertEqual(msg_type, MSG_TYPE_STATUS) + self.assertEqual(msg, exp_msg) + + @patch("select.select", autospec=True) + def test_non_retryable_error(self, m_select): + m_select.side_effect = iter([ + ([self.sck], [], []), + ]) + err = socket.error() + err.errno = errno.E2BIG + self.sck.recv.side_effect = err + msg_gen = self.reader.new_messages(timeout=1) + self.assertRaises(socket.error, next, msg_gen) + + @patch("select.select", autospec=True) + def test_timeout(self, m_select): + m_select.side_effect = iter([ + ([], [], []), + ]) + msg_gen = self.reader.new_messages(timeout=1) + self.assertRaises(StopIteration, next, msg_gen) + self.assertFalse(self.sck.recv.called) + + @patch("select.select", autospec=True) + def test_shutdown(self, m_select): + self.sck.recv.return_value = "" + msg_gen = self.reader.new_messages(timeout=None) + self.assertRaises(SocketClosed, next, msg_gen) diff --git a/calico/etcdutils.py b/calico/etcdutils.py index e6771212d5..bbd99121a8 100644 --- a/calico/etcdutils.py +++ b/calico/etcdutils.py @@ -1,4 +1,5 @@ # Copyright (c) Metaswitch Networks 2015. All rights reserved. +from collections import namedtuple import logging import re @@ -54,6 +55,12 @@ def register(self, path, on_set=None, on_del=None): node["delete"] = on_del def handle_event(self, response): + """ + :param EtcdEvent|EtcdResponse: Either a python-etcd response object + for a watch response or an instance of our dedicated EtcdEvent + class, which we use when deserialising an event that came over + the etcd driver socket. + """ _log.debug("etcd event %s for key %s", response.action, response.key) key_parts = response.key.strip("/").split("/") self._handle(key_parts, response, self.handler_root, {}) @@ -80,6 +87,9 @@ def _handle(self, key_parts, response, handler_node, captures): action, response.key, handler_node) +EtcdEvent = namedtuple("EtcdEvent", ["action", "key", "value"]) + + class EtcdClientOwner(object): """ Base class for objects that own an etcd Client. Supports diff --git a/calico/felix/config.py b/calico/felix/config.py index 816656640b..14a9fcf860 100644 --- a/calico/felix/config.py +++ b/calico/felix/config.py @@ -185,6 +185,9 @@ def __init__(self, config_path): "an endpoint to the host.", "DROP") self.add_parameter("LogFilePath", "Path to log file", "/var/log/calico/felix.log") + self.add_parameter("EtcdDriverLogFilePath", + "Path to log file for etcd driver", + "/var/log/calico/felix-etcd.log") self.add_parameter("LogSeverityFile", "Log severity for logging to file", "INFO") self.add_parameter("LogSeveritySys", @@ -261,6 +264,7 @@ def _finish_update(self, final=False): self.DEFAULT_INPUT_CHAIN_ACTION = \ self.parameters["DefaultEndpointToHostAction"].value self.LOGFILE = self.parameters["LogFilePath"].value + self.DRIVERLOGFILE = self.parameters["EtcdDriverLogFilePath"].value self.LOGLEVFILE = self.parameters["LogSeverityFile"].value self.LOGLEVSYS = self.parameters["LogSeveritySys"].value self.LOGLEVSCR = self.parameters["LogSeverityScreen"].value @@ -280,7 +284,8 @@ def _finish_update(self, final=False): common.complete_logging(self.LOGFILE, self.LOGLEVFILE, self.LOGLEVSYS, - self.LOGLEVSCR) + self.LOGLEVSCR, + gevent_in_use=True) if final: # Log configuration - the whole lot of it. @@ -386,10 +391,12 @@ def _validate_cfg(self, final=True): raise ConfigException("Invalid log level", self.parameters["LogSeverityScreen"]) - # Log file may be "None" (the literal string, case insensitive). In + # Log files may be "None" (the literal string, case insensitive). In # this case no log file should be written. if self.LOGFILE.lower() == "none": self.LOGFILE = None + if self.DRIVERLOGFILE.lower() == "none": + self.DRIVERLOGFILE = None if self.METADATA_IP.lower() == "none": # Metadata is not required. diff --git a/calico/felix/dispatch.py b/calico/felix/dispatch.py index 97214c9ab9..174224943f 100644 --- a/calico/felix/dispatch.py +++ b/calico/felix/dispatch.py @@ -47,6 +47,7 @@ def __init__(self, config, ip_version, iptables_updater): self.ifaces = set() self.programmed_leaf_chains = set() self._dirty = False + self._datamodel_in_sync = False @actor_message() def apply_snapshot(self, ifaces): @@ -63,6 +64,10 @@ def apply_snapshot(self, ifaces): # missing. self._dirty = True + if not self._datamodel_in_sync: + _log.info("Datamodel in sync, unblocking dispatch chain updates") + self._datamodel_in_sync = True + @actor_message() def on_endpoint_added(self, iface_name): """ @@ -101,7 +106,7 @@ def on_endpoint_removed(self, iface_name): self._dirty = True def _finish_msg_batch(self, batch, results): - if self._dirty: + if self._dirty and self._datamodel_in_sync: _log.debug("Interface mapping changed, reprogramming chains.") self._reprogram_chains() self._dirty = False diff --git a/calico/felix/endpoint.py b/calico/felix/endpoint.py index 6f91a02dd8..1582b68be0 100644 --- a/calico/felix/endpoint.py +++ b/calico/felix/endpoint.py @@ -66,6 +66,8 @@ def __init__(self, config, ip_type, # increffed. self.local_endpoint_ids = set() + self._data_model_in_sync = False + def _create(self, combined_id): """ Overrides ReferenceManager._create() @@ -87,26 +89,21 @@ def _on_object_started(self, endpoint_id, obj): obj.on_endpoint_update(ep, async=True) @actor_message() - def apply_snapshot(self, endpoints_by_id): - # Tell the dispatch chains about the local endpoints in advance so - # that we don't flap the dispatch chain at start-of-day. - local_iface_name_to_ep_id = {} - for ep_id, ep in endpoints_by_id.iteritems(): - if ep and ep_id.host == self.config.HOSTNAME and ep.get("name"): - local_iface_name_to_ep_id[ep.get("name")] = ep_id - self.dispatch_chains.apply_snapshot(local_iface_name_to_ep_id.keys(), - async=True) - # Then update/create endpoints and work out which endpoints have been - # deleted. - missing_endpoints = set(self.endpoints_by_id.keys()) - for endpoint_id, endpoint in endpoints_by_id.iteritems(): - self.on_endpoint_update(endpoint_id, endpoint, - force_reprogram=True) - missing_endpoints.discard(endpoint_id) - self._maybe_yield() - for endpoint_id in missing_endpoints: - self.on_endpoint_update(endpoint_id, None) - self._maybe_yield() + def on_datamodel_in_sync(self): + if not self._data_model_in_sync: + _log.info("%s: First time we've been in-sync with the datamodel," + "sending snapshot to DispatchChains.", self) + self._data_model_in_sync = True + + # Tell the dispatch chains about the local endpoints in advance so + # that we don't flap the dispatch chain at start-of-day. Note: + # the snapshot may contain information that is ahead of the + # state that our individual LocalEndpoint actors are sending to the + # DispatchChains actor. That is OK! The worst that can happen is + # that a LocalEndpoint undoes part of our update and then goes on + # to re-apply the update when it catches up to the snapshot. + local_ifaces = frozenset(self.endpoint_id_by_iface_name.keys()) + self.dispatch_chains.apply_snapshot(local_ifaces, async=True) @actor_message() def on_endpoint_update(self, endpoint_id, endpoint, force_reprogram=False): @@ -492,7 +489,7 @@ def _update_chains(self): def _remove_chains(self): try: self.iptables_updater.delete_chains(chain_names(self._suffix), - async=True) + async=False) except FailedSystemCall: _log.exception("Failed to delete chains for %s", self) else: diff --git a/calico/felix/felix.py b/calico/felix/felix.py index d219b992e5..13aa1b53c7 100644 --- a/calico/felix/felix.py +++ b/calico/felix/felix.py @@ -203,7 +203,7 @@ def dump_top_level_actors(log): def main(): # Initialise the logging with default parameters. - common.default_logging() + common.default_logging(gevent_in_use=True) # Create configuration, reading defaults from file if it exists. parser = optparse.OptionParser() @@ -222,7 +222,8 @@ def main(): common.complete_logging("/var/log/calico/felix.log", logging.DEBUG, logging.DEBUG, - logging.DEBUG) + logging.DEBUG, + gevent_in_use=True) except Exception: pass diff --git a/calico/felix/fetcd.py b/calico/felix/fetcd.py index 977eeb32a1..22f1bb55a3 100644 --- a/calico/felix/fetcd.py +++ b/calico/felix/fetcd.py @@ -19,13 +19,13 @@ Our API to etcd. Contains function to synchronize felix with etcd as well as reporting our status into etcd. """ -from collections import defaultdict import functools import os import random import json import logging -from calico.monotonic import monotonic_time +import socket +import subprocess from etcd import EtcdException, EtcdKeyNotFound import gevent @@ -34,21 +34,30 @@ from calico import common from calico.common import ValidationFailed, validate_ip_addr, canonicalise_ip -from calico.datamodel_v1 import (VERSION_DIR, READY_KEY, CONFIG_DIR, - RULES_KEY_RE, TAGS_KEY_RE, - dir_for_per_host_config, - PROFILE_DIR, HOST_DIR, EndpointId, POLICY_DIR, - HOST_IP_KEY_RE, IPAM_V4_CIDR_KEY_RE, - key_for_last_status, key_for_status, - FELIX_STATUS_DIR, get_endpoint_id_from_key, - dir_for_felix_status, ENDPOINT_STATUS_ERROR, - ENDPOINT_STATUS_DOWN, ENDPOINT_STATUS_UP) +from calico.datamodel_v1 import ( + VERSION_DIR, CONFIG_DIR, RULES_KEY_RE, TAGS_KEY_RE, + dir_for_per_host_config, PROFILE_DIR, HOST_DIR, EndpointId, HOST_IP_KEY_RE, + IPAM_V4_CIDR_KEY_RE, key_for_last_status, key_for_status, FELIX_STATUS_DIR, + get_endpoint_id_from_key, dir_for_felix_status, ENDPOINT_STATUS_ERROR, + ENDPOINT_STATUS_DOWN, ENDPOINT_STATUS_UP +) +from calico.etcddriver.protocol import ( + MessageReader, MSG_TYPE_INIT, MSG_TYPE_CONFIG, MSG_TYPE_RESYNC, + MSG_KEY_ETCD_URL, MSG_KEY_HOSTNAME, MSG_KEY_LOG_FILE, MSG_KEY_SEV_FILE, + MSG_KEY_SEV_SYSLOG, MSG_KEY_SEV_SCREEN, STATUS_IN_SYNC, + MSG_TYPE_CONFIG_LOADED, MSG_KEY_GLOBAL_CONFIG, MSG_KEY_HOST_CONFIG, + MSG_TYPE_UPDATE, MSG_KEY_KEY, MSG_KEY_VALUE, MessageWriter, + MSG_TYPE_STATUS, MSG_KEY_STATUS +) from calico.etcdutils import ( - EtcdClientOwner, EtcdWatcher, ResyncRequired, - delete_empty_parents) + EtcdClientOwner, delete_empty_parents, PathDispatcher, EtcdEvent +) from calico.felix.actor import Actor, actor_message -from calico.felix.futils import (intern_dict, intern_list, logging_exceptions, - iso_utc_timestamp, IPV4, IPV6) +from calico.felix.futils import ( + intern_dict, intern_list, logging_exceptions, iso_utc_timestamp, IPV4, + IPV6, StatCounter +) +from calico.monotonic import monotonic_time _log = logging.getLogger(__name__) @@ -75,16 +84,12 @@ POOL_V4_DIR = IPAM_V4_DIR + "/pool" CIDR_V4_KEY = POOL_V4_DIR + "/" -RESYNC_KEYS = [ - VERSION_DIR, - POLICY_DIR, - PROFILE_DIR, - CONFIG_DIR, - HOST_DIR, - IPAM_DIR, - IPAM_V4_DIR, - POOL_V4_DIR, -] +# Max number of events from driver process before we yield to another greenlet. +MAX_EVENTS_BEFORE_YIELD = 200 + + +# Global diagnostic counters. +_stats = StatCounter("Etcd counters") class EtcdAPI(EtcdClientOwner, Actor): @@ -156,6 +161,7 @@ def _periodically_resync(self): _log.debug("After jitter, next periodic resync will be in %.1f " "seconds.", sleep_time) gevent.sleep(sleep_time) + _stats.increment("Periodic resync") self.force_resync(reason="periodic resync", async=True) @logging_exceptions @@ -222,7 +228,7 @@ def load_config(self): """ Loads our config from etcd, should only be called once. - :return: an event which is triggered when the config has been loaded. + :return: an Event which is triggered when the config has been loaded. """ self._watcher.load_config.set() return self._watcher.configured @@ -233,7 +239,9 @@ def start_watch(self, splitter): Starts watching etcd for changes. Implicitly loads the config if it hasn't been loaded yet. """ - self._watcher.load_config.set() + assert self._watcher.load_config.is_set(), ( + "load_config() should be called before start_watch()." + ) self._watcher.splitter = splitter self._watcher.begin_polling.set() @@ -245,7 +253,7 @@ def force_resync(self, reason="unknown"): :param str reason: Optional reason to log out. """ _log.info("Forcing a resync with etcd. Reason: %s.", reason) - self._watcher.resync_after_current_poll = True + self._watcher.resync_requested = True if self._config.REPORT_ENDPOINT_STATUS: _log.info("Endpoint status reporting enabled, marking existing " @@ -261,305 +269,302 @@ def _on_worker_died(self, watch_greenlet): sys.exit(1) -class _FelixEtcdWatcher(EtcdWatcher, gevent.Greenlet): +class _FelixEtcdWatcher(gevent.Greenlet): """ - Greenlet that watches the etcd data model for changes. - - (1) Waits for the load_config event to be triggered. - (2) Connects to etcd and waits for the Ready flag to be set, - indicating the data model is consistent. - (3) Loads the config from etcd and passes it to the config object. - (4) Waits for the begin_polling Event to be triggered. - (5) Loads a complete snapshot from etcd and passes it to the - UpdateSplitter. - (6) Watches etcd for changes, sending them incrementally to the - UpdateSplitter. - (On etcd error) starts again from step (5) - - This greenlet is expected to be managed by the EtcdAPI Actor. + Greenlet that communicates with the etcd driver over a socket. + + * Does the initial handshake with the driver, sending it the init + message. + * Receives the pre-loaded config from the driver and uses that + to do Felix's one-off configuration. + * Sends the relevant config back to the driver. + * Processes the event stream from the driver, sending it on to + the splitter. + + This class is similar to the EtcdWatcher class in that it uses + a PathDispatcher to fan out updates but it doesn't own an etcd + connection of its own. """ def __init__(self, config, etcd_api, status_reporter, hosts_ipset): - super(_FelixEtcdWatcher, self).__init__(config.ETCD_ADDR, VERSION_DIR) + super(_FelixEtcdWatcher, self).__init__() self._config = config self._etcd_api = etcd_api self._status_reporter = status_reporter self.hosts_ipset = hosts_ipset - + # Whether we've been in sync with etcd at some point. + self._been_in_sync = False # Keep track of the config loaded from etcd so we can spot if it # changes. self.last_global_config = None self.last_host_config = None self.my_config_dir = dir_for_per_host_config(self._config.HOSTNAME) - # Events triggered by the EtcdAPI Actor to tell us to load the config # and start polling. These are one-way flags. self.load_config = Event() self.begin_polling = Event() - # Event that we trigger once the config is loaded. self.configured = Event() - # Polling state initialized at poll start time. self.splitter = None - - # Cache of known endpoints, used to resolve deletions of whole - # directory trees. - self.endpoint_ids_per_host = defaultdict(set) - # Next-hop IP addresses of our hosts, if populated in etcd. self.ipv4_by_hostname = {} - + # Forces a resync after the current poll if set. Safe to set from + # another thread. Automatically reset to False after the resync is + # triggered. + self.resync_requested = False + self.dispatcher = PathDispatcher() + # The Popen object for the driver. + self._driver_process = None + # Stats. + self.read_count = 0 + self.msgs_processed = 0 + self.last_rate_log_time = monotonic_time() # Register for events when values change. self._register_paths() def _register_paths(self): """ Program the dispatcher with the paths we care about. - - Since etcd gives us a single event for a recursive directory - deletion, we have to handle deletes for lots of directories that - we otherwise wouldn't care about. """ - reg = self.register_path - # Top-level directories etc. If these go away, stop polling and - # resync. - for key in RESYNC_KEYS: - reg(key, on_del=self._resync) - reg(READY_KEY, on_set=self.on_ready_flag_set, on_del=self._resync) + reg = self.dispatcher.register # Profiles and their contents. - reg(PER_PROFILE_DIR, on_del=self.on_profile_delete) reg(TAGS_KEY, on_set=self.on_tags_set, on_del=self.on_tags_delete) reg(RULES_KEY, on_set=self.on_rules_set, on_del=self.on_rules_delete) - # Hosts, workloads and endpoints. - reg(PER_HOST_DIR, on_del=self.on_host_delete) + # Hosts and endpoints. reg(HOST_IP_KEY, on_set=self.on_host_ip_set, on_del=self.on_host_ip_delete) - reg(WORKLOAD_DIR, on_del=self.on_host_delete) - reg(PER_ORCH_DIR, on_del=self.on_orch_delete) - reg(PER_WORKLOAD_DIR, on_del=self.on_workload_delete) - reg(ENDPOINT_DIR, on_del=self.on_workload_delete) reg(PER_ENDPOINT_KEY, on_set=self.on_endpoint_set, on_del=self.on_endpoint_delete) reg(CIDR_V4_KEY, on_set=self.on_ipam_v4_pool_set, on_del=self.on_ipam_v4_pool_delete) - # Configuration keys. If any of these is changed or set a resync is - # done, including a full reload of configuration. If any field has - # actually changed (as opposed to being reset to the same value or - # explicitly set to the default, say), Felix terminates allowing the - # init daemon to restart it. + # Configuration keys. If any of these is changed or created, we'll + # restart to pick up the change. reg(CONFIG_PARAM_KEY, - on_set=self._resync, - on_del=self._resync) + on_set=self._on_config_updated, + on_del=self._on_config_updated) reg(PER_HOST_CONFIG_PARAM_KEY, - on_set=self._resync, - on_del=self._resync) + on_set=self._on_host_config_updated, + on_del=self._on_host_config_updated) @logging_exceptions def _run(self): - """ - Greenlet main loop: loads the initial dump from etcd and then - monitors for changes and feeds them to the splitter. - """ + # Don't do anything until we're told to load the config. + _log.info("Waiting for load_config event...") self.load_config.wait() - self.loop() - - def _on_pre_resync(self): - self.wait_for_ready(RETRY_DELAY) - # Always reload the config. This lets us detect if the config has - # changed and restart felix if so. - self._load_config() - if not self.configured.is_set(): - # Unblock anyone who's waiting on the config. - self.configured.set() - if not self.begin_polling.is_set(): - _log.info("etcd worker about to wait for begin_polling event") - self.begin_polling.wait() + _log.info("...load_config set. Starting driver read %s loop", self) + # Start the driver process and wait for it to connect back to our + # socket. + self._msg_reader, self._msg_writer = self._start_driver() + # Loop reading from the socket and processing messages. + self._loop_reading_from_driver() + + def _loop_reading_from_driver(self): + while True: + for msg_type, msg in self._msg_reader.new_messages(timeout=1): + self._dispatch_msg_from_driver(msg_type, msg) + if self.resync_requested: + _log.info("Resync requested, sending resync request to driver") + self.resync_requested = False + self._msg_writer.send_message(MSG_TYPE_RESYNC) + # Check that the driver hasn't died. The recv() call should + # raise an exception when the buffer runs dry but this usually + # gets hit first. + driver_rc = self._driver_process.poll() + if driver_rc is not None: + _log.critical("Driver process died with RC = %s. Felix must " + "exit.", driver_rc) + die_and_restart() + + def _dispatch_msg_from_driver(self, msg_type, msg): + # Optimization: put update first in the "switch" block because + # it's on the critical path. + if msg_type == MSG_TYPE_UPDATE: + _stats.increment("Update messages from driver") + self._on_update_from_driver(msg) + elif msg_type == MSG_TYPE_CONFIG_LOADED: + _stats.increment("Config loaded messages from driver") + self._on_config_loaded_from_driver(msg) + elif msg_type == MSG_TYPE_STATUS: + _stats.increment("Status messages from driver") + self._on_status_from_driver(msg) + else: + raise RuntimeError("Unexpected message %s" % msg) + self.msgs_processed += 1 + if self.msgs_processed % MAX_EVENTS_BEFORE_YIELD == 0: + # Yield to ensure that other actors make progress. (gevent only + # yields for us if the socket would block.) The sleep must be + # non-zero to work around gevent issue where we could be + # immediately rescheduled. + gevent.sleep(0.000001) - def _load_config(self): + def _on_update_from_driver(self, msg): """ - Loads our configuration from etcd. Does not return - until the config is successfully loaded. + Called when the driver sends us a key/value pair update. - The first call to this method populates the config object. + After the initial handshake, the stream of events consists + entirely of updates unless something happens to change the + state of the driver. - Subsequent calls check the config hasn't changed and kill - the process if it has. This allows us to be restarted by - the init daemon in order to pick up the new config. + :param dict msg: The message received from the driver. """ - while True: - try: - global_cfg = self.client.read(CONFIG_DIR, - recursive=True) - global_dict = _build_config_dict(global_cfg) + assert self.configured.is_set(), "Received update before config" + # The driver starts polling immediately, make sure we block until + # everyone else is ready to receive updates. + self.begin_polling.wait() + # Unpack the message. + key = msg[MSG_KEY_KEY] + value = msg[MSG_KEY_VALUE] + _log.debug("Update from driver: %s -> %s", key, value) + # Output some very coarse stats. + self.read_count += 1 + if self.read_count % 1000 == 0: + now = monotonic_time() + delta = now - self.last_rate_log_time + _log.info("Processed %s updates from driver " + "%.1f/s", self.read_count, 1000.0 / delta) + self.last_rate_log_time = now + # Wrap the update in an EtcdEvent object so we can dispatch it via the + # PathDispatcher. + n = EtcdEvent("set" if value is not None else "delete", key, value) + self.dispatcher.handle_event(n) + + def _on_config_loaded_from_driver(self, msg): + """ + Called when we receive a config loaded message from the driver. + + This message is expected once per resync, when the config is + pre-loaded by the driver. + + On the first call, responds to the driver synchronously with a + config response. + + If the config has changed since a previous call, triggers Felix + to die. + """ + global_config = msg[MSG_KEY_GLOBAL_CONFIG] + host_config = msg[MSG_KEY_HOST_CONFIG] + _log.info("Config loaded by driver:\n" + "Global: %s\nPer-host: %s", + global_config, + host_config) + if self.configured.is_set(): + # We've already been configured. We don't yet support + # dynamic config update so instead we check if the config + # has changed and die if it has. + _log.info("Checking configuration for changes...") + if (host_config != self.last_host_config or + global_config != self.last_global_config): + _log.warning("Felix configuration has changed, " + "felix must restart.") + _log.info("Old host config: %s", self.last_host_config) + _log.info("New host config: %s", host_config) + _log.info("Old global config: %s", + self.last_global_config) + _log.info("New global config: %s", global_config) + die_and_restart() + else: + # First time loading the config. Report it to the config + # object. Take copies because report_etcd_config is + # destructive. + self.last_host_config = host_config.copy() + self.last_global_config = global_config.copy() + self._config.report_etcd_config(host_config, + global_config) + # Config now fully resolved, inform the driver. + driver_log_file = self._config.DRIVERLOGFILE + self._msg_writer.send_message( + MSG_TYPE_CONFIG, + { + MSG_KEY_LOG_FILE: driver_log_file, + MSG_KEY_SEV_FILE: self._config.LOGLEVFILE, + MSG_KEY_SEV_SCREEN: self._config.LOGLEVSCR, + MSG_KEY_SEV_SYSLOG: self._config.LOGLEVSYS, + } + ) + self.configured.set() - try: - host_cfg = self.client.read(self.my_config_dir, - recursive=True) - host_dict = _build_config_dict(host_cfg) - except EtcdKeyNotFound: - # It is not an error for there to be no per-host - # config; default to empty. - _log.info("No configuration overrides for this node") - host_dict = {} - except (EtcdKeyNotFound, EtcdException) as e: - # Note: we don't log the stack trace because it's too - # spammy and adds little. - _log.error("Failed to read config. etcd may be down or " - "the data model may not be ready: %r. Will " - "retry.", e) - gevent.sleep(RETRY_DELAY) - else: - if self.configured.is_set(): - # We've already been configured. We don't yet support - # dynamic config update so instead we check if the config - # has changed and die if it has. - _log.info("Checking configuration for changes...") - if (host_dict != self.last_host_config or - global_dict != self.last_global_config): - _log.warning("Felix configuration has changed, " - "felix must restart.") - _log.info("Old host config: %s", self.last_host_config) - _log.info("New host config: %s", host_dict) - _log.info("Old global config: %s", - self.last_global_config) - _log.info("New global config: %s", global_dict) - die_and_restart() - else: - # First time loading the config. Report it to the config - # object. Take copies because report_etcd_config is - # destructive. - self.last_host_config = host_dict.copy() - self.last_global_config = global_dict.copy() - self._config.report_etcd_config(host_dict, global_dict) - return - - def _on_snapshot_loaded(self, etcd_snapshot_response): + def _on_status_from_driver(self, msg): """ - Loads a snapshot from etcd and passes it to the update splitter. + Called when we receive a status update from the driver. - :raises ResyncRequired: if the Ready flag is not set in the snapshot. - """ - rules_by_id = {} - tags_by_id = {} - endpoints_by_id = {} - ipv4_pools_by_id = {} - self.endpoint_ids_per_host.clear() - self.ipv4_by_hostname.clear() - still_ready = False - for child in etcd_snapshot_response.children: - profile_id, rules = parse_if_rules(child) - if profile_id: - rules_by_id[profile_id] = rules - continue - profile_id, tags = parse_if_tags(child) - if profile_id: - tags_by_id[profile_id] = tags - continue - endpoint_id, endpoint = parse_if_endpoint(self._config, child) - if endpoint_id and endpoint: - endpoints_by_id[endpoint_id] = endpoint - self.endpoint_ids_per_host[endpoint_id.host].add(endpoint_id) - continue - pool_id, pool = parse_if_ipam_v4_pool(child) - if pool_id and pool: - ipv4_pools_by_id[pool_id] = pool - continue - if self._config.IP_IN_IP_ENABLED: - hostname, ip = parse_if_host_ip(child) - if hostname and ip: - self.ipv4_by_hostname[hostname] = ip - continue - - # Double-check the flag hasn't changed since we read it before. - if child.key == READY_KEY: - if child.value == "true": - still_ready = True - else: - _log.warning("Aborting resync because ready flag was" - "unset since we read it.") - raise ResyncRequired() - - if not still_ready: - _log.warn("Aborting resync; ready flag no longer present.") - raise ResyncRequired() - - # We now know exactly which endpoints are on this host, use that to - # clean up any endpoint statuses that should now be gone. - our_endpoints_ids = self.endpoint_ids_per_host[self._config.HOSTNAME] - self.clean_up_endpoint_statuses(our_endpoints_ids) - - # Actually apply the snapshot. This does not return anything, but - # just sends the relevant messages to the relevant threads to make - # all the processing occur. - _log.info("Snapshot parsed, passing to update splitter") - self.splitter.apply_snapshot(rules_by_id, - tags_by_id, - endpoints_by_id, - ipv4_pools_by_id, - async=True) - if self._config.IP_IN_IP_ENABLED: - # We only support IPv4 for host tracking right now so there's not - # much point in going via the splitter. - # FIXME Support IP-in-IP for IPv6. - _log.info("Sending (%d) host IPs to ipset.", - len(self.ipv4_by_hostname)) - self.hosts_ipset.replace_members(self.ipv4_by_hostname.values(), - async=True) - - def clean_up_endpoint_statuses(self, our_endpoints_ids): - """ - Mark any endpoint status reports for non-existent endpoints - for cleanup. + The driver sends us status messages whenever its status changes. + It moves through these states: - :param set our_endpoints_ids: Set of endpoint IDs for endpoints on - this host. - """ - if not self._config.REPORT_ENDPOINT_STATUS: - _log.debug("Endpoint status reporting disabled, ignoring.") - return + (1) wait-for-ready (waiting for the global ready flag to become set) + (2) resync (resyncing with etcd, processing a snapshot and any + concurrent events) + (3) in-sync (snapshot processsing complete, now processing only events + from etcd) - our_host_dir = "/".join([FELIX_STATUS_DIR, self._config.HOSTNAME, - "workload"]) - try: - # Grab all the existing status reports. - response = self.client.read(our_host_dir, - recursive=True) - except EtcdKeyNotFound: - _log.info("No endpoint statuses found, nothing to clean up") - else: - for node in response.leaves: - combined_id = get_endpoint_id_from_key(node.key) - if combined_id and combined_id not in our_endpoints_ids: - # We found an endpoint in our status reporting tree that - # wasn't in the main tree. Mark it as dirty so the status - # reporting thread will clean it up. - _log.debug("Endpoint %s removed by resync, marking " - "status key for cleanup", - combined_id) - self._status_reporter.mark_endpoint_dirty(combined_id, - async=True) - elif node.dir: - # This leaf is an empty directory, try to clean it up. - # This is safe even if another thread is adding keys back - # into the directory. - _log.debug("Found empty directory %s, cleaning up", - node.key) - delete_empty_parents(self.client, node.key, our_host_dir) + If the driver falls out of sync with etcd then it will start again + from (1). - def _resync(self, response, **kwargs): + If the status is in-sync, triggers the relevant processing. """ - Force a resync. - :raises ResyncRequired: always. + status = msg[MSG_KEY_STATUS] + _log.info("etcd driver status changed to %s", status) + if status == STATUS_IN_SYNC and not self._been_in_sync: + # We're now in sync, tell the Actors that need to do start-of-day + # cleanup. + self.begin_polling.wait() # Make sure splitter is set. + self._been_in_sync = True + self.splitter.on_datamodel_in_sync(async=True) + if self._config.REPORT_ENDPOINT_STATUS: + self._status_reporter.clean_up_endpoint_statuses(async=True) + self._update_hosts_ipset() + + def _start_driver(self): """ - _log.warning("Resync triggered due to change to %s", response.key) - raise ResyncRequired() + Starts the driver subprocess, connects to it over the socket + and sends it the init message. - def on_ready_flag_set(self, response): - if response.value != "true": - raise ResyncRequired() + Stores the Popen object in self._driver_process for future + access. + + :return: the connected socket to the driver. + """ + _log.info("Creating server socket.") + try: + os.unlink("/run/felix-driver.sck") + except OSError: + _log.debug("Failed to delete driver socket, assuming it " + "didn't exist.") + update_socket = socket.socket(socket.AF_UNIX, + socket.SOCK_STREAM) + update_socket.bind("/run/felix-driver.sck") + update_socket.listen(1) + self._driver_process = subprocess.Popen([sys.executable, + "-m", + "calico.etcddriver", + "/run/felix-driver.sck"]) + _log.info("Started etcd driver with PID %s", self._driver_process.pid) + update_conn, _ = update_socket.accept() + _log.info("Accepted connection on socket") + # No longer need the server socket, remove it. + try: + os.unlink("/run/felix-driver.sck") + except OSError: + # Unexpected but carry on... + _log.exception("Failed to unlink socket") + else: + _log.info("Unlinked server socket") + + # Wrap the socket in reader/writer objects that simplify using the + # protocol. + reader = MessageReader(update_conn) + writer = MessageWriter(update_conn) + # Give the driver its config. + writer.send_message( + MSG_TYPE_INIT, + { + MSG_KEY_ETCD_URL: "http://" + self._config.ETCD_ADDR, + MSG_KEY_HOSTNAME: self._config.HOSTNAME, + } + ) + return reader, writer def on_endpoint_set(self, response, hostname, orchestrator, workload_id, endpoint_id): @@ -567,7 +572,7 @@ def on_endpoint_set(self, response, hostname, orchestrator, combined_id = EndpointId(hostname, orchestrator, workload_id, endpoint_id) _log.debug("Endpoint %s updated", combined_id) - self.endpoint_ids_per_host[combined_id.host].add(combined_id) + _stats.increment("Endpoint created/updated") endpoint = parse_endpoint(self._config, combined_id, response.value) self.splitter.on_endpoint_update(combined_id, endpoint, async=True) @@ -577,14 +582,13 @@ def on_endpoint_delete(self, response, hostname, orchestrator, combined_id = EndpointId(hostname, orchestrator, workload_id, endpoint_id) _log.debug("Endpoint %s deleted", combined_id) - self.endpoint_ids_per_host[combined_id.host].discard(combined_id) - if not self.endpoint_ids_per_host[combined_id.host]: - del self.endpoint_ids_per_host[combined_id.host] + _stats.increment("Endpoint deleted") self.splitter.on_endpoint_update(combined_id, None, async=True) def on_rules_set(self, response, profile_id): """Handler for rules updates, passes the update to the splitter.""" _log.debug("Rules for %s set", profile_id) + _stats.increment("Rules created/updated") rules = parse_rules(profile_id, response.value) profile_id = intern(profile_id.encode("utf8")) self.splitter.on_rules_update(profile_id, rules, async=True) @@ -592,11 +596,13 @@ def on_rules_set(self, response, profile_id): def on_rules_delete(self, response, profile_id): """Handler for rules deletes, passes the update to the splitter.""" _log.debug("Rules for %s deleted", profile_id) + _stats.increment("Rules deleted") self.splitter.on_rules_update(profile_id, None, async=True) def on_tags_set(self, response, profile_id): """Handler for tags updates, passes the update to the splitter.""" _log.debug("Tags for %s set", profile_id) + _stats.increment("Tags created/updated") rules = parse_tags(profile_id, response.value) profile_id = intern(profile_id.encode("utf8")) self.splitter.on_tags_update(profile_id, rules, async=True) @@ -604,37 +610,15 @@ def on_tags_set(self, response, profile_id): def on_tags_delete(self, response, profile_id): """Handler for tags deletes, passes the update to the splitter.""" _log.debug("Tags for %s deleted", profile_id) + _stats.increment("Tags deleted") self.splitter.on_tags_update(profile_id, None, async=True) - def on_profile_delete(self, response, profile_id): - """ - Handler for a whole profile deletion - - Fakes a tag and rules delete. - """ - # Fake deletes for the rules and tags. - _log.debug("Whole profile %s deleted", profile_id) - self.splitter.on_rules_update(profile_id, None, async=True) - self.splitter.on_tags_update(profile_id, None, async=True) - - def on_host_delete(self, response, hostname): - """ - Handler for deletion of a whole host directory. - - Deletes all the contained endpoints. - """ - ids_on_that_host = self.endpoint_ids_per_host.pop(hostname, set()) - _log.info("Host %s deleted, removing %d endpoints", - hostname, len(ids_on_that_host)) - for endpoint_id in ids_on_that_host: - self.splitter.on_endpoint_update(endpoint_id, None, async=True) - self.on_host_ip_delete(response, hostname) - def on_host_ip_set(self, response, hostname): if not self._config.IP_IN_IP_ENABLED: _log.debug("Ignoring update to %s because IP-in-IP is disabled", response.key) return + _stats.increment("Host IP created/updated") ip = parse_host_ip(hostname, response.value) if ip: self.ipv4_by_hostname[hostname] = ip @@ -642,60 +626,52 @@ def on_host_ip_set(self, response, hostname): _log.warning("Invalid IP for hostname %s: %s, treating as " "deletion", hostname, response.value) self.ipv4_by_hostname.pop(hostname, None) - self.hosts_ipset.replace_members(self.ipv4_by_hostname.values(), - async=True) + self._update_hosts_ipset() def on_host_ip_delete(self, response, hostname): if not self._config.IP_IN_IP_ENABLED: _log.debug("Ignoring update to %s because IP-in-IP is disabled", response.key) return + _stats.increment("Host IP deleted") if self.ipv4_by_hostname.pop(hostname, None): - self.hosts_ipset.replace_members(self.ipv4_by_hostname.values(), - async=True) + self._update_hosts_ipset() + + def _update_hosts_ipset(self): + if not self._been_in_sync: + _log.debug("Deferring update to hosts ipset until we're in-sync") + return + self.hosts_ipset.replace_members(self.ipv4_by_hostname.values(), + async=True) + + def _on_config_updated(self, response, config_param): + new_value = response.value + if self.last_global_config.get(config_param) != new_value: + _log.critical("Global config value %s updated. Felix must be " + "restarted.", config_param) + die_and_restart() + _stats.increment("Global config (non) updates") + + def _on_host_config_updated(self, response, hostname, config_param): + if hostname != self._config.HOSTNAME: + _log.debug("Ignoring config update for host %s", hostname) + return + _stats.increment("Per-host config created/updated") + new_value = response.value + if self.last_host_config.get(config_param) != new_value: + _log.critical("Global config value %s updated. Felix must be " + "restarted.", config_param) + die_and_restart() def on_ipam_v4_pool_set(self, response, pool_id): + _stats.increment("IPAM pool created/updated") pool = parse_ipam_pool(pool_id, response.value) self.splitter.on_ipam_pool_update(pool_id, pool, async=True) def on_ipam_v4_pool_delete(self, response, pool_id): + _stats.increment("IPAM pool deleted") self.splitter.on_ipam_pool_update(pool_id, None, async=True) - def on_orch_delete(self, response, hostname, orchestrator): - """ - Handler for deletion of a whole host orchestrator directory. - - Deletes all the contained endpoints. - """ - _log.info("Orchestrator dir %s/%s deleted, removing contained hosts", - hostname, orchestrator) - orchestrator = intern(orchestrator.encode("utf8")) - for endpoint_id in list(self.endpoint_ids_per_host[hostname]): - if endpoint_id.orchestrator == orchestrator: - self.splitter.on_endpoint_update(endpoint_id, None, async=True) - self.endpoint_ids_per_host[hostname].discard(endpoint_id) - if not self.endpoint_ids_per_host[hostname]: - del self.endpoint_ids_per_host[hostname] - - def on_workload_delete(self, response, hostname, orchestrator, - workload_id): - """ - Handler for deletion of a whole workload directory. - - Deletes all the contained endpoints. - """ - _log.debug("Workload dir %s/%s/%s deleted, removing endpoints", - hostname, orchestrator, workload_id) - orchestrator = intern(orchestrator.encode("utf8")) - workload_id = intern(workload_id.encode("utf8")) - for endpoint_id in list(self.endpoint_ids_per_host[hostname]): - if (endpoint_id.orchestrator == orchestrator and - endpoint_id.workload == workload_id): - self.splitter.on_endpoint_update(endpoint_id, None, async=True) - self.endpoint_ids_per_host[hostname].discard(endpoint_id) - if not self.endpoint_ids_per_host[hostname]: - del self.endpoint_ids_per_host[hostname] - class EtcdStatusReporter(EtcdClientOwner, Actor): """ @@ -716,6 +692,7 @@ def __init__(self, config): self._newer_dirty_endpoints = set() self._older_dirty_endpoints = set() + self._cleanup_pending = False self._timer_scheduled = False self._reporting_allowed = True @@ -723,8 +700,10 @@ def __init__(self, config): def on_endpoint_status_changed(self, endpoint_id, ip_type, status): assert isinstance(endpoint_id, EndpointId) if status is not None: + _stats.increment("Endpoint status deleted") self._endpoint_status[ip_type][endpoint_id] = status else: + _stats.increment("Endpoint status updated") self._endpoint_status[ip_type].pop(endpoint_id, None) self._mark_endpoint_dirty(endpoint_id) @@ -744,10 +723,6 @@ def _on_timer_pop(self): self._timer_scheduled = False self._reporting_allowed = True - @actor_message() - def mark_endpoint_dirty(self, endpoint_id): - self._mark_endpoint_dirty(endpoint_id) - def _mark_endpoint_dirty(self, endpoint_id): assert isinstance(endpoint_id, EndpointId) if endpoint_id in self._older_dirty_endpoints: @@ -760,15 +735,34 @@ def _mark_endpoint_dirty(self, endpoint_id): _log.debug("Marking endpoint %s dirty", endpoint_id) self._newer_dirty_endpoints.add(endpoint_id) + @actor_message() + def clean_up_endpoint_statuses(self): + """ + Note that we need to do cleanup. We'll then try/retry from + _finish_msg_batch(). + """ + self._cleanup_pending = True + def _finish_msg_batch(self, batch, results): if not self._config.REPORT_ENDPOINT_STATUS: - _log.error("StatusReporter called even though status reporting " - "disabled. Ignoring.") + _log.warning("StatusReporter called even though status reporting " + "disabled. Ignoring.") self._endpoint_status[IPV4].clear() self._endpoint_status[IPV6].clear() self._newer_dirty_endpoints.clear() self._older_dirty_endpoints.clear() return + + if self._cleanup_pending: + try: + self._attempt_cleanup() + except EtcdException as e: + _log.error("Cleanup failed: %r", e) + _stats.increment("Status report cleanup failed") + else: + _stats.increment("Status report cleanup done") + self._cleanup_pending = False + if self._reporting_allowed: # We're not rate limited, go ahead and do a write to etcd. _log.debug("Status reporting is allowed by rate limit.") @@ -794,20 +788,49 @@ def _finish_msg_batch(self, batch, results): # Reset the rate limit flag. self._reporting_allowed = False - if not self._timer_scheduled and not self._reporting_allowed: - # Schedule a timer to stop our rate limiting. + if not self._timer_scheduled and ((not self._reporting_allowed) or + self._cleanup_pending): + # Schedule a timer to stop our rate limiting or retry cleanup. timeout = self._config.ENDPOINT_REPORT_DELAY - timeout *= 0.9 + (random.random() * 0.2) # Jitter by +/- 10%. + timeout *= (0.9 + (random.random() * 0.2)) # Jitter by +/- 10%. gevent.spawn_later(timeout, self._on_timer_pop, async=True) self._timer_scheduled = True + def _attempt_cleanup(self): + our_host_dir = "/".join([FELIX_STATUS_DIR, self._config.HOSTNAME, + "workload"]) + try: + # Grab all the existing status reports. + response = self.client.read(our_host_dir, + recursive=True) + except EtcdKeyNotFound: + _log.info("No endpoint statuses found, nothing to clean up") + else: + # Mark all statuses we find as dirty. This will result in any + # unknown endpoints being cleaned up. + for node in response.leaves: + combined_id = get_endpoint_id_from_key(node.key) + if combined_id: + _log.debug("Endpoint %s removed by resync, marking " + "status key for cleanup", + combined_id) + self._mark_endpoint_dirty(combined_id) + elif node.dir: + # This leaf is an empty directory, try to clean it up. + # This is safe even if another thread is adding keys back + # into the directory. + _log.debug("Found empty directory %s, cleaning up", + node.key) + delete_empty_parents(self.client, node.key, our_host_dir) + def _write_endpoint_status_to_etcd(self, ep_id, status): """ Try to actually write the status dict into etcd or delete the key if it is no longer needed. """ + _stats.increment("Per-port status report etcd writes") status_key = ep_id.path_for_status if status: _log.debug("Writing endpoint status %s = %s", ep_id, status) @@ -859,19 +882,6 @@ def die_and_restart(): os._exit(1) -def _build_config_dict(cfg_node): - """ - Updates the config dict provided from the given etcd node, which - should point at a config directory. - """ - config_dict = {} - for child in cfg_node.children: - key = child.key.rsplit("/").pop() - value = str(child.value) - config_dict[key] = value - return config_dict - - # Intern JSON keys as we load them to reduce occupancy. FIELDS_TO_INTERN = set([ # Endpoint dicts. It doesn't seem worth interning items like the MAC @@ -894,20 +904,6 @@ def _build_config_dict(cfg_node): ) -def parse_if_endpoint(config, etcd_node): - combined_id = get_endpoint_id_from_key(etcd_node.key) - if combined_id: - # Got an endpoint. - if etcd_node.action == "delete": - _log.debug("Found deleted endpoint %s", combined_id) - endpoint = None - else: - endpoint = parse_endpoint(config, combined_id, etcd_node.value) - # EndpointId does the interning for us. - return combined_id, endpoint - return None, None - - def parse_endpoint(config, combined_id, raw_json): endpoint = safe_decode_json(raw_json, log_tag="endpoint %s" % combined_id.endpoint) @@ -915,26 +911,13 @@ def parse_endpoint(config, combined_id, raw_json): common.validate_endpoint(config, combined_id, endpoint) except ValidationFailed as e: _log.warning("Validation failed for endpoint %s, treating as " - "missing: %s", combined_id, e.message) + "missing: %s; %r", combined_id, e.message, raw_json) endpoint = None else: _log.debug("Validated endpoint : %s", endpoint) return endpoint -def parse_if_rules(etcd_node): - m = RULES_KEY_RE.match(etcd_node.key) - if m: - # Got some rules. - profile_id = m.group("profile_id") - if etcd_node.action == "delete": - rules = None - else: - rules = parse_rules(profile_id, etcd_node.value) - return intern(profile_id.encode("utf8")), rules - return None, None - - def parse_rules(profile_id, raw_json): rules = safe_decode_json(raw_json, log_tag="rules %s" % profile_id) try: @@ -947,19 +930,6 @@ def parse_rules(profile_id, raw_json): return rules -def parse_if_tags(etcd_node): - m = TAGS_KEY_RE.match(etcd_node.key) - if m: - # Got some tags. - profile_id = m.group("profile_id") - if etcd_node.action == "delete": - tags = None - else: - tags = parse_tags(profile_id, etcd_node.value) - return intern(profile_id.encode("utf8")), tags - return None, None - - def parse_tags(profile_id, raw_json): tags = safe_decode_json(raw_json, log_tag="tags %s" % profile_id) try: @@ -974,19 +944,6 @@ def parse_tags(profile_id, raw_json): return intern_list(tags) -def parse_if_host_ip(etcd_node): - m = HOST_IP_KEY_RE.match(etcd_node.key) - if m: - # Got some rules. - hostname = m.group("hostname") - if etcd_node.action == "delete": - ip = None - else: - ip = parse_host_ip(hostname, etcd_node.value) - return hostname, ip - return None, None - - def parse_host_ip(hostname, raw_value): if raw_value is None or validate_ip_addr(raw_value): return canonicalise_ip(raw_value, None) @@ -995,19 +952,6 @@ def parse_host_ip(hostname, raw_value): return None -def parse_if_ipam_v4_pool(etcd_node): - m = IPAM_V4_CIDR_KEY_RE.match(etcd_node.key) - if m: - # Got some rules. - pool_id = m.group("encoded_cidr") - if etcd_node.action == "delete": - pool = None - else: - pool = parse_ipam_pool(pool_id, etcd_node.value) - return pool_id, pool - return None, None - - def parse_ipam_pool(pool_id, raw_json): pool = safe_decode_json(raw_json, log_tag="ipam pool %s" % pool_id) try: @@ -1027,4 +971,3 @@ def safe_decode_json(raw_json, log_tag=None): _log.warning("Failed to decode JSON for %s: %r. Returning None.", log_tag, raw_json) return None - diff --git a/calico/felix/ipsets.py b/calico/felix/ipsets.py index bd4aeafd65..36ace4aaf6 100644 --- a/calico/felix/ipsets.py +++ b/calico/felix/ipsets.py @@ -72,7 +72,7 @@ def __init__(self, ip_type, config): # index-update functions. We apply the updates in _finish_msg_batch(). # May include non-live tag IDs. self._dirty_tags = set() - self._force_reprogram = False + self._datamodel_in_sync = False def _create(self, tag_id): active_ipset = TagIpset(futils.uniquely_shorten(tag_id, 16), @@ -80,6 +80,14 @@ def _create(self, tag_id): max_elem=self._config.MAX_IPSET_SIZE) return active_ipset + def _maybe_start(self, obj_id): + if self._datamodel_in_sync: + _log.debug("Datamodel is in-sync, deferring to superclass.") + return super(IpsetManager, self)._maybe_start(obj_id) + else: + _log.info("Delaying startup of tag %s because datamodel is" + "not in sync.", obj_id) + def _on_object_started(self, tag_id, active_ipset): _log.debug("TagIpset actor for %s started", tag_id) # Fill the ipset in with its members, this will trigger its first @@ -96,11 +104,10 @@ def _update_active_ipset(self, tag_id): :param tag_id: The ID of the tag, must be an active tag. """ assert self._is_starting_or_live(tag_id) + assert self._datamodel_in_sync active_ipset = self.objects_by_id[tag_id] members = frozenset(self.ip_owners_by_tag.get(tag_id, {}).iterkeys()) - active_ipset.replace_members(members, - force_reprogram=self._force_reprogram, - async=True) + active_ipset.replace_members(members, async=True) def _update_dirty_active_ipsets(self): """ @@ -108,11 +115,14 @@ def _update_dirty_active_ipsets(self): Clears the set of dirty tags as a side-effect. """ + num_updates = 0 for tag_id in self._dirty_tags: if self._is_starting_or_live(tag_id): self._update_active_ipset(tag_id) + num_updates += 1 self._maybe_yield() - _log.info("Sent updates to %s updated tags", len(self._dirty_tags)) + if num_updates > 0: + _log.info("Sent updates to %s updated tags", num_updates) self._dirty_tags.clear() @property @@ -121,41 +131,11 @@ def nets_key(self): return nets @actor_message() - def apply_snapshot(self, tags_by_prof_id, endpoints_by_id): - """ - Apply a snapshot read from etcd, replacing existing state. - - :param tags_by_prof_id: A dict mapping security profile ID to a list of - profile tags. - :param endpoints_by_id: A dict mapping EndpointId objects to endpoint - data dicts. - """ - _log.info("Applying tags snapshot. %s tags, %s endpoints", - len(tags_by_prof_id), len(endpoints_by_id)) - missing_profile_ids = set(self.tags_by_prof_id.keys()) - for profile_id, tags in tags_by_prof_id.iteritems(): - assert tags is not None - self.on_tags_update(profile_id, tags) - missing_profile_ids.discard(profile_id) - self._maybe_yield() - for profile_id in missing_profile_ids: - self.on_tags_update(profile_id, None) - self._maybe_yield() - del missing_profile_ids - missing_endpoints = set(self.endpoint_data_by_ep_id.keys()) - for endpoint_id, endpoint in endpoints_by_id.iteritems(): - assert endpoint is not None - endpoint_data = self._endpoint_data_from_dict(endpoint_id, - endpoint) - self._on_endpoint_data_update(endpoint_id, endpoint_data) - missing_endpoints.discard(endpoint_id) - self._maybe_yield() - for endpoint_id in missing_endpoints: - self._on_endpoint_data_update(endpoint_id, EMPTY_ENDPOINT_DATA) - self._maybe_yield() - self._force_reprogram = True - _log.info("Tags snapshot applied: %s tags, %s endpoints", - len(tags_by_prof_id), len(endpoints_by_id)) + def on_datamodel_in_sync(self): + if not self._datamodel_in_sync: + _log.info("Datamodel now in sync, uncorking updates to TagIpsets") + self._datamodel_in_sync = True + self._maybe_start_all() @actor_message() def cleanup(self): @@ -450,7 +430,6 @@ def _finish_msg_batch(self, batch, results): """ super(IpsetManager, self)._finish_msg_batch(batch, results) self._update_dirty_active_ipsets() - self._force_reprogram = False class EndpointData(object): @@ -554,7 +533,7 @@ def replace_members(self, members, force_reprogram=False): """ Replace the members of this ipset with the supplied set. - :param set[str]|list[str] members: IP address strings. Must be a copy + :param set[str] members: IP address strings. Must be a copy (as this routine keeps a link to it). """ _log.info("Replacing members of ipset %s", self.name) diff --git a/calico/felix/profilerules.py b/calico/felix/profilerules.py index 4e47e5df6b..ea1d746114 100644 --- a/calico/felix/profilerules.py +++ b/calico/felix/profilerules.py @@ -44,6 +44,7 @@ def __init__(self, ip_version, iptables_updater, ipset_manager): self.iptables_updater = iptables_updater self.ipset_manager = ipset_manager self.rules_by_profile_id = {} + self._datamodel_in_sync = False def _create(self, profile_id): return ProfileRules(profile_id, @@ -57,18 +58,32 @@ def _on_object_started(self, profile_id, active_profile): profile_or_none) active_profile.on_profile_update(profile_or_none, async=True) + def _maybe_start(self, obj_id, in_sync=False): + """ + Override: gates starting the ProfileRules on being in sync. + + :param obj_id: The ID of the object (profile) that we'd like to start. + :param in_sync: True if we know that this profile is in-sync even if + we might not have received the global in-sync message. + """ + in_sync |= self._datamodel_in_sync + if in_sync or obj_id in self.rules_by_profile_id: + # Either we're globally in-sync or we've explicitly heard about + # this profile so we know it is in sync. Defer to the superclass. + _log.debug("Profile %s is in-sync, deferring to superclass.", + obj_id) + return super(RulesManager, self)._maybe_start(obj_id) + else: + _log.info("Delaying startup of profile %s because datamodel is" + "not in sync.", obj_id) + @actor_message() - def apply_snapshot(self, rules_by_profile_id): - _log.info("Rules manager applying snapshot; %s rules", - len(rules_by_profile_id)) - missing_ids = set(self.rules_by_profile_id.keys()) - for profile_id, profile in rules_by_profile_id.iteritems(): - self.on_rules_update(profile_id, profile, - force_reprogram=True) # Skips queue - missing_ids.discard(profile_id) - self._maybe_yield() - for dead_profile_id in missing_ids: - self.on_rules_update(dead_profile_id, None) + def on_datamodel_in_sync(self): + if not self._datamodel_in_sync: + _log.info("%s: datamodel now in sync, unblocking profile startup", + self) + self._datamodel_in_sync = True + self._maybe_start_all() @actor_message() def on_rules_update(self, profile_id, profile, force_reprogram=False): @@ -84,6 +99,12 @@ def on_rules_update(self, profile_id, profile, force_reprogram=False): ap = self.objects_by_id[profile_id] ap.on_profile_update(profile, force_reprogram=force_reprogram, async=True) + elif profile_id in self.objects_by_id: + _log.debug("Checking if the update allows us to start profile %s", + profile_id) + # Pass in_sync=True because we now explicitly know this profile is + # in sync, even if this is a deletion. + self._maybe_start(profile_id, in_sync=True) class ProfileRules(RefCountedActor): @@ -168,7 +189,7 @@ def _finish_msg_batch(self, batch, results): _log.info("%s unreferenced, removing our chains", self) self._delete_chains() self._ipset_refs.discard_all() - self._ipset_refs = None # Break ref cycle. + self._ipset_refs = None # Break ref cycle. self._profile = None self._pending_profile = None finally: @@ -212,7 +233,8 @@ def _finish_msg_batch(self, batch, results): self.id) else: self._dirty = False - elif not self._ipset_refs.ready: + else: + assert not self._ipset_refs.ready _log.info("Can't program rules %s yet, waiting on ipsets", self.id) diff --git a/calico/felix/refcount.py b/calico/felix/refcount.py index ff75913312..012d80d75a 100644 --- a/calico/felix/refcount.py +++ b/calico/felix/refcount.py @@ -141,10 +141,24 @@ def on_object_cleanup_complete(self, object_id, obj): # May have unblocked start of new object... self._maybe_start(object_id) + def _maybe_start_all(self): + _log.debug("Checking all objects to see if they can be started") + for obj_id in self.objects_by_id: + self._maybe_start(obj_id) + def _maybe_start(self, obj_id): """ Starts the actor with the given ID if it is present and there are no pending cleanups for that ID. + + Subclasses may override this method to place additional + pre-requisites on starting the object. They should call + this implementation if they are happy for the start to + proceed. + + If the subclass chooses to block startup, it must later call + this method (or the convenience method _maybe_start_all()) + when it wants to allow startup to proceed. """ obj = self.objects_by_id.get(obj_id) if (obj and @@ -196,9 +210,8 @@ def _create(self, object_id): raise NotImplementedError() # pragma nocover def _is_starting_or_live(self, obj_id): - return (obj_id in self.objects_by_id - and self.objects_by_id[obj_id].ref_mgmt_state in - (STARTING, LIVE)) + return (obj_id in self.objects_by_id and + self.objects_by_id[obj_id].ref_mgmt_state in (STARTING, LIVE)) class RefHelper(object): @@ -366,4 +379,4 @@ def _notify_cleanup_complete(self): is complete. Notifies the manager. """ _log.debug("Notifying manager that %s is done cleaning up", self) - self._manager.on_object_cleanup_complete(self._id, self, async=True) \ No newline at end of file + self._manager.on_object_cleanup_complete(self._id, self, async=True) diff --git a/calico/felix/splitter.py b/calico/felix/splitter.py index da56b6901a..96b05ae340 100644 --- a/calico/felix/splitter.py +++ b/calico/felix/splitter.py @@ -51,51 +51,15 @@ def __init__(self, config, ipsets_mgrs, rules_managers, endpoint_managers, self._cleanup_scheduled = False @actor_message() - def apply_snapshot(self, rules_by_prof_id, tags_by_prof_id, - endpoints_by_id, ipv4_pools_by_id): + def on_datamodel_in_sync(self): """ - Replaces the whole cache state with the input. Applies deltas vs the - current active state. - - :param rules_by_prof_id: A dict mapping security profile ID to a list - of profile rules, each of which is a dict. - :param tags_by_prof_id: A dict mapping security profile ID to a list of - profile tags. - :param endpoints_by_id: A dict mapping EndpointId objects to endpoint - data dicts. - :param ipv4_pools_by_id: A dict mapping IPAM pool ID to dicts - representing the pool. + Called when the data-model is known to be in-sync. """ - # Step 1: fire in data update events to the profile and tag managers - # so they can build their indexes before we activate anything. - _log.info("Applying snapshot. Queueing rules.") - for rules_mgr in self.rules_mgrs: - rules_mgr.apply_snapshot(rules_by_prof_id, async=True) - _log.info("Applying snapshot. Queueing tags/endpoints to ipset mgr.") - for ipset_mgr in self.ipsets_mgrs: - ipset_mgr.apply_snapshot(tags_by_prof_id, endpoints_by_id, - async=True) - - # Step 2: fire in update events into the endpoint manager, which will - # recursively trigger activation of profiles and tags. - _log.info("Applying snapshot. Queueing endpoints->endpoint mgr.") - for ep_mgr in self.endpoint_mgrs: - ep_mgr.apply_snapshot(endpoints_by_id, async=True) - - # Step 3: send update to NAT manager. - _log.info("Applying snapshot. Queueing IPv4 pools -> masq mgr.") - self.ipv4_masq_manager.apply_snapshot(ipv4_pools_by_id, async=True) - - _log.info("Applying snapshot. DONE. %s rules, %s tags, " - "%s endpoints, %s pools", len(rules_by_prof_id), - len(tags_by_prof_id), len(endpoints_by_id), - len(ipv4_pools_by_id)) - - # Since we don't wait for all the above processing to finish, set a - # timer to clean up orphaned ipsets and tables later. If the snapshot - # takes longer than this timer to apply then we might do the cleanup - # before the snapshot is finished. That would cause dropped packets - # until applying the snapshot finishes. + for mgr in self.ipsets_mgrs + self.rules_mgrs + self.endpoint_mgrs: + mgr.on_datamodel_in_sync(async=True) + + # Now we're in sync, give the managers some time to get their house in + # order, then trigger the start-of-day cleanup. if not self._cleanup_scheduled: _log.info("No cleanup scheduled, scheduling one.") gevent.spawn_later(self.config.STARTUP_CLEANUP_DELAY, diff --git a/calico/felix/test/base.py b/calico/felix/test/base.py index f4bd319ff9..972e6fc71a 100644 --- a/calico/felix/test/base.py +++ b/calico/felix/test/base.py @@ -26,6 +26,8 @@ import mock +mock.patch.object = getattr(mock.patch, "object") # Keep PyCharm linter happy. + _log = logging.getLogger(__name__) @@ -73,4 +75,8 @@ def __eq__(self, other): return False def __repr__(self): - return '%s(%r)' % (self.__class__.__name__, self.json_obj) \ No newline at end of file + return '%s(%r)' % (self.__class__.__name__, self.json_obj) + + +class ExpectedException(Exception): + pass \ No newline at end of file diff --git a/calico/felix/test/data/felix_nolog.cfg b/calico/felix/test/data/felix_nolog.cfg index a14188ff33..704229a038 100644 --- a/calico/felix/test/data/felix_nolog.cfg +++ b/calico/felix/test/data/felix_nolog.cfg @@ -1,3 +1,4 @@ [log] # Log file path. LogFilePath = none +EtcdDriverLogFilePath = none diff --git a/calico/felix/test/test_actor.py b/calico/felix/test/test_actor.py index 59fb3dbbe1..d56189426c 100644 --- a/calico/felix/test/test_actor.py +++ b/calico/felix/test/test_actor.py @@ -19,17 +19,17 @@ Tests of the Actor framework. """ -import logging -import itertools import gc +import itertools +import logging import sys -from gevent.event import AsyncResult import mock -from calico.felix.actor import actor_message, ResultOrExc, SplitBatchAndRetry -from calico.felix.test.base import BaseTestCase -from calico.felix import actor +from gevent.event import AsyncResult +from calico.felix import actor +from calico.felix.actor import actor_message, ResultOrExc, SplitBatchAndRetry +from calico.felix.test.base import BaseTestCase, ExpectedException # Logger log = logging.getLogger(__name__) @@ -184,7 +184,7 @@ def test_own_batch(self): def test_blocking_call(self): self._actor.start() # Really start it. self._actor.do_a(async=False) - self.assertRaises(ExpectedException, self._actor.do_exc, async=False) + self.assertRaises(ExpectedException, self._actor.do_exc, async=False) def test_same_actor_call(self): """ @@ -369,10 +369,6 @@ def on_unreferenced(self, async=None): return self.on_unref_result -class ExpectedException(Exception): - pass - - class FinishException(Exception): pass diff --git a/calico/felix/test/test_config.py b/calico/felix/test/test_config.py index f84ccaa460..7a1e85e9f6 100644 --- a/calico/felix/test/test_config.py +++ b/calico/felix/test/test_config.py @@ -118,6 +118,7 @@ def test_no_logfile(self): config.report_etcd_config({}, cfg_dict) self.assertEqual(config.LOGFILE, None) + self.assertEqual(config.DRIVERLOGFILE, None) def test_no_metadata(self): # Metadata can be excluded by explicitly saying "none" diff --git a/calico/felix/test/test_endpoint.py b/calico/felix/test/test_endpoint.py index b5a589b4f7..719957f75b 100644 --- a/calico/felix/test/test_endpoint.py +++ b/calico/felix/test/test_endpoint.py @@ -20,7 +20,7 @@ """ from contextlib import nested import logging -from calico.felix.endpoint import EndpointManager +from calico.felix.endpoint import EndpointManager, LocalEndpoint from calico.felix.fetcd import EtcdAPI, EtcdStatusReporter from calico.felix.fiptables import IptablesUpdater from calico.felix.dispatch import DispatchChains @@ -39,6 +39,133 @@ _log = logging.getLogger(__name__) +mock.patch.object = getattr(mock.patch, "object") # Keep PyCharm linter happy. + +ENDPOINT_ID = EndpointId("hostname", "b", "c", "d") + + +class TestEndpointManager(BaseTestCase): + def setUp(self): + super(TestEndpointManager, self).setUp() + self.m_config = Mock(spec=config.Config) + self.m_config.HOSTNAME = "hostname" + self.m_updater = Mock(spec=IptablesUpdater) + self.m_dispatch = Mock(spec=DispatchChains) + self.m_rules_mgr = Mock(spec=RulesManager) + self.m_status_reporter = Mock(spec=EtcdStatusReporter) + self.mgr = EndpointManager(self.m_config, "IPv4", self.m_updater, + self.m_dispatch, self.m_rules_mgr, + self.m_status_reporter) + self.mgr.get_and_incref = Mock() + self.mgr.decref = Mock() + + def test_create(self): + obj = self.mgr._create(ENDPOINT_ID) + self.assertTrue(isinstance(obj, LocalEndpoint)) + + def test_on_started(self): + ep = {"name": "tap1234"} + self.mgr.on_endpoint_update(ENDPOINT_ID, + ep, + async=True) + self.step_actor(self.mgr) + m_endpoint = Mock(spec=LocalEndpoint) + self.mgr._on_object_started(ENDPOINT_ID, m_endpoint) + self.assertEqual( + m_endpoint.on_endpoint_update.mock_calls, + [mock.call(ep, async=True)] + ) + + def test_on_datamodel_in_sync(self): + ep = {"name": "tap1234"} + self.mgr.on_endpoint_update(ENDPOINT_ID, + ep, + async=True) + self.step_actor(self.mgr) + self.mgr.on_datamodel_in_sync(async=True) + self.step_actor(self.mgr) + self.assertEqual( + self.m_dispatch.apply_snapshot.mock_calls, + [mock.call(frozenset(["tap1234"]), async=True)] + ) + # Second call should have no effect. + self.m_dispatch.apply_snapshot.reset_mock() + self.mgr.on_datamodel_in_sync(async=True) + self.step_actor(self.mgr) + self.assertEqual(self.m_dispatch.apply_snapshot.mock_calls, []) + + def test_endpoint_update_not_our_host(self): + ep = {"name": "tap1234"} + with mock.patch.object(self.mgr, "_is_starting_or_live") as m_sol: + self.mgr.on_endpoint_update(EndpointId("notus", "b", "c", "d"), + ep, + async=True) + self.step_actor(self.mgr) + self.assertFalse(m_sol.called) + + def test_endpoint_live_obj(self): + ep = {"name": "tap1234"} + # First send in an update to trigger creation. + self.mgr.on_endpoint_update(ENDPOINT_ID, ep, async=True) + self.step_actor(self.mgr) + self.assertEqual(self.mgr.get_and_incref.mock_calls, + [mock.call(ENDPOINT_ID)]) + m_endpoint = Mock(spec=LocalEndpoint) + self.mgr.objects_by_id[ENDPOINT_ID] = m_endpoint + # Then send a second update to check that it gets passed on to the + # LocalEndpoint. + with mock.patch.object(self.mgr, "_is_starting_or_live") as m_sol: + m_sol.return_value = True + self.mgr.on_endpoint_update(ENDPOINT_ID, ep, async=True) + self.step_actor(self.mgr) + self.assertEqual(m_sol.mock_calls, [mock.call(ENDPOINT_ID)]) + self.assertEqual(m_endpoint.on_endpoint_update.mock_calls, + [mock.call(ep, force_reprogram=False, + async=True)]) + self.assertTrue(ENDPOINT_ID in self.mgr.local_endpoint_ids) + # Finally, send in a deletion. + m_endpoint.on_endpoint_update.reset_mock() + with mock.patch.object(self.mgr, "_is_starting_or_live") as m_sol: + m_sol.return_value = True + self.mgr.on_endpoint_update(ENDPOINT_ID, None, async=True) + self.step_actor(self.mgr) + self.assertEqual(m_endpoint.on_endpoint_update.mock_calls, + [mock.call(None, force_reprogram=False, + async=True)]) + self.assertEqual(self.mgr.decref.mock_calls, [mock.call(ENDPOINT_ID)]) + self.assertFalse(ENDPOINT_ID in self.mgr.local_endpoint_ids) + + def test_on_interface_update_unknown(self): + with mock.patch.object(self.mgr, "_is_starting_or_live") as m_sol: + self.mgr.on_interface_update("foo", True, async=True) + self.step_actor(self.mgr) + self.assertFalse(m_sol.called) + + def test_on_interface_update_known(self): + ep = {"name": "tap1234"} + m_endpoint = Mock(spec=LocalEndpoint) + self.mgr.objects_by_id[ENDPOINT_ID] = m_endpoint + with mock.patch.object(self.mgr, "_is_starting_or_live") as m_sol: + m_sol.return_value = True + self.mgr.on_endpoint_update(ENDPOINT_ID, ep, async=True) + self.mgr.on_interface_update("tap1234", True, async=True) + self.step_actor(self.mgr) + self.assertEqual( + m_endpoint.on_interface_update.mock_calls, + [mock.call(True, async=True)] + ) + + def test_on_interface_update_known_but_not_live(self): + ep = {"name": "tap1234"} + m_endpoint = Mock(spec=LocalEndpoint) + self.mgr.objects_by_id[ENDPOINT_ID] = m_endpoint + with mock.patch.object(self.mgr, "_is_starting_or_live") as m_sol: + m_sol.return_value = False + self.mgr.on_endpoint_update(ENDPOINT_ID, ep, async=True) + self.mgr.on_interface_update("tap1234", True, async=True) + self.step_actor(self.mgr) + self.assertEqual(m_endpoint.on_interface_update.mock_calls, []) + class TestLocalEndpoint(BaseTestCase): def setUp(self): diff --git a/calico/felix/test/test_fetcd.py b/calico/felix/test/test_fetcd.py index 094735c05d..4ee5279399 100644 --- a/calico/felix/test/test_fetcd.py +++ b/calico/felix/test/test_fetcd.py @@ -12,6 +12,8 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +import socket +import subprocess from datetime import datetime import json import logging @@ -19,19 +21,27 @@ from etcd import EtcdResult, EtcdException import etcd from gevent.event import Event +import gevent from mock import Mock, call, patch, ANY from calico.datamodel_v1 import EndpointId +from calico.etcddriver.protocol import MessageReader, MessageWriter, \ + MSG_TYPE_CONFIG_LOADED, MSG_TYPE_STATUS, STATUS_RESYNC, MSG_KEY_STATUS, \ + MSG_TYPE_UPDATE, MSG_KEY_KEY, MSG_KEY_VALUE, MSG_KEY_TYPE, \ + MSG_KEY_HOST_CONFIG, MSG_KEY_GLOBAL_CONFIG, MSG_TYPE_CONFIG, \ + MSG_KEY_LOG_FILE, MSG_KEY_SEV_FILE, MSG_KEY_SEV_SCREEN, MSG_KEY_SEV_SYSLOG, \ + STATUS_IN_SYNC from calico.felix.config import Config from calico.felix.futils import IPV4, IPV6 from calico.felix.ipsets import IpsetActor -from calico.felix.fetcd import (_FelixEtcdWatcher, ResyncRequired, EtcdAPI, +from calico.felix.fetcd import (_FelixEtcdWatcher, EtcdAPI, die_and_restart, EtcdStatusReporter, combine_statuses) from calico.felix.splitter import UpdateSplitter from calico.felix.test.base import BaseTestCase, JSONString _log = logging.getLogger(__name__) +patch.object = getattr(patch, "object") # Keep PyCharm linter happy. VALID_ENDPOINT = { "state": "active", @@ -60,71 +70,85 @@ class TestEtcdAPI(BaseTestCase): - - @patch("calico.felix.fetcd._FelixEtcdWatcher", autospec=True) - @patch("gevent.spawn", autospec=True) - def test_create(self, m_spawn, m_etcd_watcher): - m_config = Mock(spec=Config) - m_config.ETCD_ADDR = ETCD_ADDRESS - m_hosts_ipset = Mock(spec=IpsetActor) - api = EtcdAPI(m_config, m_hosts_ipset) - m_etcd_watcher.assert_has_calls([ - call(m_config, m_hosts_ipset).link(api._on_worker_died), - call(m_config, m_hosts_ipset).start(), + def setUp(self): + super(TestEtcdAPI, self).setUp() + self.m_config = Mock(spec=Config) + self.m_config.ETCD_ADDR = ETCD_ADDRESS + self.m_hosts_ipset = Mock(spec=IpsetActor) + with patch("calico.felix.fetcd._FelixEtcdWatcher", + autospec=True) as m_etcd_watcher: + with patch("gevent.spawn", autospec=True) as m_spawn: + self.api = EtcdAPI(self.m_config, self.m_hosts_ipset) + self.m_spawn = m_spawn + self.m_etcd_watcher = m_etcd_watcher.return_value + self.m_etcd_watcher.load_config = Mock(spec=Event) + self.m_etcd_watcher.begin_polling = Mock(spec=Event) + self.m_etcd_watcher.configured = Mock(spec=Event) + + def test_create(self): + self.m_etcd_watcher.assert_has_calls([ + call.link(self.api._on_worker_died), + call.start(), ]) - m_spawn.assert_has_calls([ - call(api._periodically_resync), - call(api._periodically_resync).link_exception(api._on_worker_died) + self.m_spawn.assert_has_calls([ + call(self.api._periodically_resync), + call(self.api._periodically_resync).link_exception( + self.api._on_worker_died) ]) - @patch("calico.felix.fetcd._FelixEtcdWatcher", autospec=True) - @patch("gevent.spawn", autospec=True) @patch("gevent.sleep", autospec=True) - def test_periodic_resync_mainline(self, m_sleep, m_spawn, m_etcd_watcher): + def test_periodic_resync_mainline(self, m_sleep): + self.m_config.RESYNC_INTERVAL = 10 m_configured = Mock(spec=Event) - m_etcd_watcher.return_value.configured = m_configured - m_config = Mock(spec=Config) - m_config.ETCD_ADDR = ETCD_ADDRESS - m_hosts_ipset = Mock(spec=IpsetActor) - api = EtcdAPI(m_config, m_hosts_ipset) - m_config.RESYNC_INTERVAL = 10 - with patch.object(api, "force_resync") as m_force_resync: + self.m_etcd_watcher.configured = m_configured + with patch.object(self.api, "force_resync") as m_force_resync: m_force_resync.side_effect = ExpectedException() - self.assertRaises(ExpectedException, api._periodically_resync) + self.assertRaises(ExpectedException, + self.api._periodically_resync) m_configured.wait.assert_called_once_with() m_sleep.assert_called_once_with(ANY) sleep_time = m_sleep.call_args[0][0] self.assertTrue(sleep_time >= 10) self.assertTrue(sleep_time <= 12) - @patch("calico.felix.fetcd._FelixEtcdWatcher", autospec=True) - @patch("gevent.spawn", autospec=True) @patch("gevent.sleep", autospec=True) - def test_periodic_resync_disabled(self, m_sleep, m_spawn, m_etcd_watcher): - m_etcd_watcher.return_value.configured = Mock(spec=Event) - m_config = Mock(spec=Config) - m_config.ETCD_ADDR = ETCD_ADDRESS - m_hosts_ipset = Mock(spec=IpsetActor) - api = EtcdAPI(m_config, m_hosts_ipset) - m_config.RESYNC_INTERVAL = 0 - with patch.object(api, "force_resync") as m_force_resync: + def test_periodic_resync_disabled(self, m_sleep): + self.m_config.RESYNC_INTERVAL = 0 + self.m_etcd_watcher.configured = Mock(spec=Event) + with patch.object(self.api, "force_resync") as m_force_resync: m_force_resync.side_effect = Exception() - api._periodically_resync() - - @patch("calico.felix.fetcd._FelixEtcdWatcher", autospec=True) - @patch("gevent.spawn", autospec=True) - def test_force_resync(self, m_spawn, m_etcd_watcher): - m_config = Mock(spec=Config) - m_config.ETCD_ADDR = ETCD_ADDRESS - m_config.REPORT_ENDPOINT_STATUS = True - m_hosts_ipset = Mock(spec=IpsetActor) - api = EtcdAPI(m_config, m_hosts_ipset) - endpoint_id = EndpointId("foo", "bar", "baz", "biff") - with patch.object(api, "status_reporter") as m_status_rep: - api.force_resync(async=True) - self.step_actor(api) + self.api._periodically_resync() + + def test_force_resync(self): + self.m_config.REPORT_ENDPOINT_STATUS = True + with patch.object(self.api, "status_reporter") as m_status_rep: + self.api.force_resync(async=True) + self.step_actor(self.api) m_status_rep.resync.assert_called_once_with(async=True) - self.assertTrue(m_etcd_watcher.return_value.resync_after_current_poll) + self.assertTrue(self.m_etcd_watcher.resync_requested) + + def test_load_config(self): + result = self.api.load_config(async=True) + self.step_actor(self.api) + conf = result.get() + self.assertEqual(conf, self.m_etcd_watcher.configured) + self.m_etcd_watcher.load_config.set.assert_called_once_with() + + def test_start_watch(self): + m_splitter = Mock() + self.api.load_config(async=True) + result = self.api.start_watch(m_splitter, async=True) + self.step_actor(self.api) + self.m_etcd_watcher.load_config.set.assert_called_once_with() + self.assertEqual(self.m_etcd_watcher.splitter, m_splitter) + self.m_etcd_watcher.begin_polling.set.assert_called_once_with() + + @patch("sys.exit", autospec=True) + def test_on_worker_died(self, m_exit): + glet = gevent.spawn(lambda: None) + glet.link(self.api._on_worker_died) + glet.join(1) + m_exit.assert_called_once_with(1) class ExpectedException(Exception): @@ -148,114 +172,193 @@ def setUp(self): self.m_hosts_ipset) self.m_splitter = Mock(spec=UpdateSplitter) self.watcher.splitter = self.m_splitter - self.client = Mock() - self.watcher.client = self.client + self.m_reader = Mock(spec=MessageReader) + self.m_writer = Mock(spec=MessageWriter) + self.watcher._msg_reader = self.m_reader + self.watcher._msg_writer = self.m_writer + self.m_driver_proc = Mock(spec=subprocess.Popen) + self.watcher._driver_process = self.m_driver_proc + + def test_run(self): + with patch.object(self.watcher.load_config, "wait") as m_wait: + with patch.object(self.watcher, "_start_driver") as m_start: + m_reader = Mock() + m_writer = Mock() + m_start.return_value = (m_reader, m_writer) + m_reader.new_messages.side_effect = ExpectedException() + self.assertRaises(ExpectedException, self.watcher._run) + self.assertEqual(m_wait.mock_calls, [call()]) - @patch("gevent.sleep", autospec=True) - @patch("calico.felix.fetcd._build_config_dict", autospec=True) @patch("calico.felix.fetcd.die_and_restart", autospec=True) - def test_load_config(self, m_die, m_build_dict, m_sleep): - # First call, loads the config. - global_cfg = {"foo": "bar"} - m_build_dict.side_effect = iter([ - # First call, global-only. - global_cfg, - # Second call, no change. - global_cfg, - # Third call, change of config. - {"foo": "baz"}, {"biff": "bop"}]) - self.client.read.side_effect = iter([ - # First time round the loop, fail to read global config, should - # retry. - etcd.EtcdKeyNotFound, - # Then get the global config but there's not host-only config. - None, etcd.EtcdKeyNotFound, - # Twice... - None, etcd.EtcdKeyNotFound, - # Then some host-only config shows up. - None, None]) - - # First call. - self.watcher._load_config() - - m_sleep.assert_called_once_with(5) - self.assertFalse(m_die.called) - - m_report = self.m_config.report_etcd_config - rpd_host_cfg, rpd_global_cfg = m_report.mock_calls[0][1] - self.assertEqual(rpd_host_cfg, {}) - self.assertEqual(rpd_global_cfg, global_cfg) - self.assertTrue(rpd_host_cfg is not self.watcher.last_host_config) - self.assertTrue(rpd_global_cfg is not self.watcher.last_global_config) - self.assertEqual(rpd_host_cfg, self.watcher.last_host_config) - self.assertEqual(rpd_global_cfg, self.watcher.last_global_config) - - self.assertEqual(self.watcher.last_host_config, {}) - self.assertEqual(self.watcher.last_global_config, global_cfg) - self.watcher.configured.set() # Normally done by the caller. - self.client.read.assert_has_calls([ - call("/calico/v1/config", recursive=True), - call("/calico/v1/host/hostname/config", recursive=True), + def test_read_loop(self, m_die): + self.m_reader.new_messages.side_effect = iter([ + iter([]), + iter([(MSG_TYPE_STATUS, {MSG_KEY_STATUS: STATUS_RESYNC})]) ]) + self.m_driver_proc.poll.side_effect = iter([ + None, 1 + ]) + m_die.side_effect = ExpectedException() + with patch.object(self.watcher, "_dispatch_msg_from_driver") as m_disp: + self.assertRaises(ExpectedException, + self.watcher._loop_reading_from_driver) + self.assertEqual(m_disp.mock_calls, + [call(MSG_TYPE_STATUS, + {MSG_KEY_STATUS: STATUS_RESYNC})]) - # Second call, no change. - self.watcher._load_config() - self.assertFalse(m_die.called) - - # Third call, should detect the config change and die. - self.watcher._load_config() - m_die.assert_called_once_with() - - def test_on_snapshot_loaded(self): - m_response = Mock() - - endpoint_on_host = Mock() - endpoint_on_host.key = ("/calico/v1/host/hostname/workload/" - "orch/wlid/endpoint/epid") - endpoint_on_host.value = ENDPOINT_STR - - bad_endpoint_on_host = Mock() - bad_endpoint_on_host.key = ("/calico/v1/host/hostname/workload/" - "orch/wlid/endpoint/epid2") - bad_endpoint_on_host.value = ENDPOINT_STR[:10] - - endpoint_not_on_host = Mock() - endpoint_not_on_host.key = ("/calico/v1/host/other/workload/" - "orch/wlid/endpoint/epid") - endpoint_not_on_host.value = ENDPOINT_STR - - still_ready = Mock() - still_ready.key = ("/calico/v1/Ready") - still_ready.value = "true" - - m_response.children = [ - endpoint_on_host, - bad_endpoint_on_host, - endpoint_not_on_host, - still_ready, - ] - with patch.object(self.watcher, - "clean_up_endpoint_statuses") as m_clean: - self.watcher._on_snapshot_loaded(m_response) + @patch("calico.felix.fetcd.die_and_restart", autospec=True) + def test_read_loop_resync(self, m_die): + self.m_reader.new_messages.side_effect = iter([iter([]), iter([])]) + self.m_driver_proc.poll.side_effect = iter([None, 1]) + self.watcher.resync_requested = True + m_die.side_effect = ExpectedException() + self.assertRaises(ExpectedException, + self.watcher._loop_reading_from_driver) + + def test_dispatch_from_driver(self): + for msg_type, expected_method in [ + (MSG_TYPE_UPDATE, "_on_update_from_driver"), + (MSG_TYPE_CONFIG_LOADED, "_on_config_loaded_from_driver"), + (MSG_TYPE_STATUS, "_on_status_from_driver"),]: + with patch.object(self.watcher, expected_method) as m_meth: + msg = Mock() + self.watcher._dispatch_msg_from_driver(msg_type, msg) + self.assertEqual(m_meth.mock_calls, [call(msg)]) + + def test_dispatch_from_driver_unexpected(self): + self.assertRaises(RuntimeError, + self.watcher._dispatch_msg_from_driver, + "unknown", {}) + + @patch("gevent.sleep") + def test_dispatch_yield(self, m_sleep): + for _ in xrange(399): + with patch.object(self.watcher, "_on_update_from_driver") as m_upd: + msg = Mock() + self.watcher._dispatch_msg_from_driver(MSG_TYPE_UPDATE, msg) + self.assertEqual(m_sleep.mock_calls, [call(0.000001)]) + + def test_on_update_from_driver(self): + self.watcher.read_count = 999 + self.watcher.configured.set() + with patch.object(self.watcher, "begin_polling") as m_begin: + self.watcher._on_update_from_driver({ + MSG_KEY_TYPE: MSG_TYPE_UPDATE, + MSG_KEY_KEY: "/calico/v1/Ready", + MSG_KEY_VALUE: "true", + }) + m_begin.wait.assert_called_once_with() - # Cleanup should only get the endpoints on our host. - m_clean.assert_called_once_with( - set([EndpointId("hostname", "orch", "wlid", "epid")]) + @patch("calico.felix.fetcd.die_and_restart", autospec=True) + def test_on_config_loaded(self, m_die): + self.m_config.DRIVERLOGFILE = "/tmp/driver.log" + global_config = {"InterfacePrefix": "tap"} + local_config = {"LogSeverityFile": "DEBUG"} + self.watcher._on_config_loaded_from_driver({ + MSG_KEY_GLOBAL_CONFIG: global_config, + MSG_KEY_HOST_CONFIG: local_config, + }) + self.assertTrue(self.watcher.configured.is_set()) + self.assertEqual( + self.m_config.report_etcd_config.mock_calls, + [call(local_config, global_config)] ) + self.assertEqual( + self.m_writer.send_message.mock_calls, + [call(MSG_TYPE_CONFIG, + { + MSG_KEY_LOG_FILE: "/tmp/driver.log", + MSG_KEY_SEV_FILE: self.m_config.LOGLEVFILE, + MSG_KEY_SEV_SCREEN: self.m_config.LOGLEVSCR, + MSG_KEY_SEV_SYSLOG: self.m_config.LOGLEVSYS, + })] + ) + self.assertEqual(m_die.mock_calls, []) + + # Check a subsequent config change results in Felix dying. + global_config = {"InterfacePrefix": "not!tap"} + local_config = {"LogSeverityFile": "not!DEBUG"} + self.watcher._on_config_loaded_from_driver({ + MSG_KEY_GLOBAL_CONFIG: global_config, + MSG_KEY_HOST_CONFIG: local_config, + }) + self.assertEqual(m_die.mock_calls, [call()]) + + def test_on_status_from_driver(self): + self.watcher._on_status_from_driver({ + MSG_KEY_STATUS: STATUS_RESYNC + }) + self.assertFalse(self.watcher._been_in_sync) + + with patch.object(self.watcher, "begin_polling") as m_begin: + # Two calls but second should be ignored... + self.watcher._on_status_from_driver({ + MSG_KEY_STATUS: STATUS_IN_SYNC + }) + self.watcher._on_status_from_driver({ + MSG_KEY_STATUS: STATUS_IN_SYNC + }) + m_begin.wait.assert_called_once_with() + self.assertTrue(self.watcher._been_in_sync) + self.assertEqual(self.m_splitter.on_datamodel_in_sync.mock_calls, + [call(async=True)]) + self.assertEqual(self.m_hosts_ipset.replace_members.mock_calls, + [call([], async=True)]) + + @patch("subprocess.Popen") + @patch("socket.socket") + @patch("os.unlink") + def test_start_driver(self, m_unlink, m_socket, m_popen): + m_sck = Mock() + m_socket.return_value = m_sck + m_conn = Mock() + m_sck.accept.return_value = m_conn, None + reader, writer = self.watcher._start_driver() + self.assertEqual(m_socket.mock_calls[0], call(socket.AF_UNIX, + socket.SOCK_STREAM)) + self.assertEqual(m_sck.bind.mock_calls, + [call("/run/felix-driver.sck")]) + self.assertEqual(m_sck.listen.mock_calls, [call(1)]) + self.assertEqual(m_popen.mock_calls[0], + call([ANY, "-m", "calico.etcddriver", + "/run/felix-driver.sck"])) + self.assertEqual(m_unlink.mock_calls, + [call("/run/felix-driver.sck")] * 2) + self.assertTrue(isinstance(reader, MessageReader)) + self.assertTrue(isinstance(writer, MessageWriter)) + + @patch("subprocess.Popen") + @patch("socket.socket") + @patch("os.unlink") + def test_start_driver_unlink_fail(self, m_unlink, m_socket, m_popen): + m_unlink.side_effect = OSError() + m_sck = Mock() + m_socket.return_value = m_sck + m_conn = Mock() + m_sck.accept.return_value = m_conn, None + reader, writer = self.watcher._start_driver() + self.assertTrue(isinstance(reader, MessageReader)) + self.assertTrue(isinstance(writer, MessageWriter)) + + def test_update_hosts_ipset_not_in_sync(self): + self.watcher._update_hosts_ipset() + self.assertEqual(self.m_hosts_ipset.mock_calls, []) - def test_resync_flag(self): - self.watcher.resync_after_current_poll = True - self.watcher.next_etcd_index = 1 - self.assertRaises(ResyncRequired, - self.watcher.wait_for_etcd_event) - self.assertFalse(self.watcher.resync_after_current_poll) + @patch("calico.felix.fetcd.die_and_restart", autospec=True) + def test_config_set(self, m_die): + self.watcher.last_global_config = {} + self.dispatch("/calico/v1/config/InterfacePrefix", + "set", value="foo") + self.assertEqual(m_die.mock_calls, [call()]) - def test_ready_flag_set(self): - self.dispatch("/calico/v1/Ready", "set", value="true") - self.assertRaises(ResyncRequired, self.dispatch, - "/calico/v1/Ready", "set", value="false") - self.assertRaises(ResyncRequired, self.dispatch, - "/calico/v1/Ready", "set", value="foo") + @patch("calico.felix.fetcd.die_and_restart", autospec=True) + def test_host_config_set(self, m_die): + self.watcher.last_host_config = {} + self.dispatch("/calico/v1/host/notourhostname/config/InterfacePrefix", + "set", value="foo") + self.dispatch("/calico/v1/host/hostname/config/InterfacePrefix", + "set", value="foo") + self.assertEqual(m_die.mock_calls, [call()]) def test_endpoint_set(self): self.dispatch("/calico/v1/host/h1/workload/o1/w1/endpoint/e1", @@ -284,60 +387,6 @@ def test_endpoint_set_invalid(self): async=True, ) - def test_parent_dir_delete(self): - """ - Test that deletions of parent directories of endpoints are - correctly handled. - """ - # This additional endpoint should be ignored by the deletes below. - self.dispatch("/calico/v1/host/h2/workload/o1/w2/endpoint/e2", - "set", value=ENDPOINT_STR) - for path in ["/calico/v1/host/h1", - "/calico/v1/host/h1/workload", - "/calico/v1/host/h1/workload/o1", - "/calico/v1/host/h1/workload/o1/w1", - "/calico/v1/host/h1/workload/o1/w1/endpoint"]: - # Create endpoints in the cache. - self.dispatch("/calico/v1/host/h1/workload/o1/w1/endpoint/e1", - "set", value=ENDPOINT_STR) - self.dispatch("/calico/v1/host/h1/workload/o1/w1/endpoint/e2", - "set", value=ENDPOINT_STR) - # This endpoint should not get cleaned up if only workload w1 is - # deleted... - self.dispatch("/calico/v1/host/h1/workload/o1/w3/endpoint/e3", - "set", value=ENDPOINT_STR) - - self.assertEqual(self.watcher.endpoint_ids_per_host, { - "h1": set([EndpointId("h1", "o1", "w1", "e1"), - EndpointId("h1", "o1", "w1", "e2"), - EndpointId("h1", "o1", "w3", "e3")]), - "h2": set([EndpointId("h2", "o1", "w2", "e2")]), - }) - self.m_splitter.on_endpoint_update.reset_mock() - # Delete one of its parent dirs, should delete the endpoint. - self.dispatch(path, "delete") - exp_calls = [ - call(EndpointId("h1", "o1", "w1", "e1"), None, async=True), - call(EndpointId("h1", "o1", "w1", "e2"), None, async=True), - ] - if path < "/calico/v1/host/h1/workload/o1/w1": - # Should also delete workload w3. - exp_calls.append(call(EndpointId("h1", "o1", "w3", "e3"), - None, async=True)) - self.m_splitter.on_endpoint_update.assert_has_calls(exp_calls, - any_order=True) - # Cache should be cleaned up. - exp_cache = {"h2": set([EndpointId("h2", "o1", "w2", "e2")])} - if path >= "/calico/v1/host/h1/workload/o1/w1": - # Should not have deleted workload w3. Add it in. - exp_cache["h1"] = set([EndpointId("h1", "o1", "w3", "e3")]) - self.assertEqual(self.watcher.endpoint_ids_per_host, exp_cache) - - # Then simulate another delete, should have no effect. - self.m_splitter.on_endpoint_update.reset_mock() - self.dispatch(path, "delete") - self.assertFalse(self.m_splitter.on_endpoint_update.called) - def test_rules_set(self): self.dispatch("/calico/v1/policy/profile/prof1/rules", "set", value=RULES_STR) @@ -380,30 +429,6 @@ def test_tags_set_invalid(self): None, async=True) - def test_dispatch_delete_resync(self): - """ - Test dispatcher is correctly configured to trigger resync for - expected paths. - """ - for key in ["/calico/v1", - "/calico/v1/host", - "/calico/v1/policy", - "/calico/v1/policy/profile", - "/calico/v1/config", - "/calico/v1/config/Foo", - "/calico/v1/Ready",]: - self.assertRaises(ResyncRequired, self.dispatch, key, "delete") - - def test_per_profile_del(self): - """ - Test profile deletion triggers deletion for tags and rules. - """ - self.dispatch("/calico/v1/policy/profile/profA", action="delete") - self.m_splitter.on_tags_update.assert_called_once_with("profA", None, - async=True) - self.m_splitter.on_rules_update.assert_called_once_with("profA", None, - async=True) - def test_tags_del(self): """ Test tag-only deletion. @@ -438,6 +463,7 @@ def test_host_ip_set(self): """ Test set for the IP of a host. """ + self.watcher._been_in_sync = True self.dispatch("/calico/v1/host/foo/bird_ip", action="set", value="10.0.0.1") self.m_hosts_ipset.replace_members.assert_called_once_with( @@ -461,6 +487,7 @@ def test_host_ip_del(self): """ Test set for the IP of a host. """ + self.watcher._been_in_sync = True self.dispatch("/calico/v1/host/foo/bird_ip", action="set", value="10.0.0.1") self.m_hosts_ipset.reset_mock() @@ -475,6 +502,7 @@ def test_host_ip_invalid(self): """ Test set for the IP of a host. """ + self.watcher._been_in_sync = True self.dispatch("/calico/v1/host/foo/bird_ip", action="set", value="10.0.0.1") self.m_hosts_ipset.reset_mock() @@ -485,25 +513,15 @@ def test_host_ip_invalid(self): async=True, ) - def test_host_del_clears_ip(self): - """ - Test set for the IP of a host. - """ - self.dispatch("/calico/v1/host/foo/bird_ip", - action="set", value="10.0.0.1") - self.m_hosts_ipset.reset_mock() - self.dispatch("/calico/v1/host/foo", - action="delete") - self.m_hosts_ipset.replace_members.assert_called_once_with( - [], - async=True, - ) + def test_ipam_pool_set(self): + self.dispatch("/calico/v1/ipam/v4/pool/1234", action="set", value="{}") + self.assertEqual(self.m_splitter.on_ipam_pool_update.mock_calls, + [call("1234", None, async=True)]) - def test_config_update_triggers_resync(self): - self.assertRaises(ResyncRequired, self.dispatch, - "/calico/v1/config/Foo", "set", "bar") - self.assertRaises(ResyncRequired, self.dispatch, - "/calico/v1/host/foo/config/Foo", "set", "bar") + def test_ipam_pool_del(self): + self.dispatch("/calico/v1/ipam/v4/pool/1234", action="delete") + self.assertEqual(self.m_splitter.on_ipam_pool_update.mock_calls, + [call("1234", None, async=True)]) @patch("os._exit", autospec=True) @patch("gevent.sleep", autospec=True) @@ -522,48 +540,6 @@ def dispatch(self, key, action, value=None): m_response.value = value self.watcher.dispatcher.handle_event(m_response) - def test_clean_up_endpoint_status(self): - self.m_config.REPORT_ENDPOINT_STATUS = True - ep_id = EndpointId("hostname", - "openstack", - "workloadid", - "endpointid") - - empty_dir = Mock() - empty_dir.key = ("/calico/felix/v1/host/hostname/workload/" - "openstack/foobar") - empty_dir.dir = True - - missing_ep = Mock() - missing_ep.key = ("/calico/felix/v1/host/hostname/workload/" - "openstack/aworkload/endpoint/anendpoint") - - self.client.read.return_value.leaves = [ - empty_dir, - missing_ep, - ] - self.watcher.clean_up_endpoint_statuses(set([ep_id])) - - # Missing endpoint should have been marked for cleanup. - self.m_status_rep.mark_endpoint_dirty.assert_called_once_with( - EndpointId("hostname", - "openstack", - "aworkload", - "anendpoint"), - async=True - ) - - def test_clean_up_endpoint_status_not_found(self): - self.m_config.REPORT_ENDPOINT_STATUS = True - self.client.read.side_effect = etcd.EtcdKeyNotFound() - self.watcher.clean_up_endpoint_statuses(set()) - self.assertFalse(self.m_status_rep.mark_endpoint_dirty.called) - - def test_clean_up_endpoint_status_disabled(self): - self.m_config.REPORT_ENDPOINT_STATUS = False - self.client.read.side_effect = self.failureException - self.watcher.clean_up_endpoint_statuses(set()) - class TestEtcdReporting(BaseTestCase): def setUp(self): @@ -729,6 +705,12 @@ def test_on_endpoint_status_mainline(self): self.assertEqual(self.rep._newer_dirty_endpoints, set()) self.assertEqual(self.rep._older_dirty_endpoints, set()) + def test_mark_endpoint_dirty_already_dirty(self): + endpoint_id = EndpointId("a", "b", "c", "d") + self.rep._older_dirty_endpoints.add(endpoint_id) + self.rep._mark_endpoint_dirty(endpoint_id) + self.assertFalse(endpoint_id in self.rep._newer_dirty_endpoints) + def test_on_endpoint_status_failure(self): # Send in an endpoint status update. endpoint_id = EndpointId("foo", "bar", "baz", "biff") @@ -829,3 +811,58 @@ def assert_combined_status(self, a, b, expected): self.assertEqual(result, expected, "Expected %r and %r to combine to %s but got %r" % (lhs, rhs, expected, result)) + + def test_clean_up_endpoint_status(self): + self.m_config.REPORT_ENDPOINT_STATUS = True + ep_id = EndpointId("foo", + "openstack", + "workloadid", + "endpointid") + + empty_dir = Mock() + empty_dir.key = ("/calico/felix/v1/host/foo/workload/" + "openstack/foobar") + empty_dir.dir = True + + missing_ep = Mock() + missing_ep.key = ("/calico/felix/v1/host/foo/workload/" + "openstack/aworkload/endpoint/anendpoint") + + self.m_client.read.return_value.leaves = [ + empty_dir, + missing_ep, + ] + with patch.object(self.rep, "_mark_endpoint_dirty") as m_mark: + self.rep.clean_up_endpoint_statuses(async=True) + self.step_actor(self.rep) + + # Missing endpoint should have been marked for cleanup. + m_mark.assert_called_once_with( + EndpointId("foo", + "openstack", + "aworkload", + "anendpoint") + ) + + def test_clean_up_endpoint_status_etcd_error(self): + self.m_config.REPORT_ENDPOINT_STATUS = True + with patch.object(self.rep, "_attempt_cleanup") as m_clean: + m_clean.side_effect = EtcdException() + self.rep.clean_up_endpoint_statuses(async=True) + self.step_actor(self.rep) + self.assertTrue(self.rep._cleanup_pending) + + def test_clean_up_endpoint_status_not_found(self): + self.m_config.REPORT_ENDPOINT_STATUS = True + self.m_client.read.side_effect = etcd.EtcdKeyNotFound() + with patch.object(self.rep, "_mark_endpoint_dirty") as m_mark: + self.rep.clean_up_endpoint_statuses(async=True) + self.step_actor(self.rep) + self.assertFalse(m_mark.called) + + def test_clean_up_endpoint_status_disabled(self): + self.m_config.REPORT_ENDPOINT_STATUS = False + self.m_client.read.side_effect = self.failureException + self.rep.clean_up_endpoint_statuses(async=True) + self.step_actor(self.rep) + diff --git a/calico/felix/test/test_fiptables.py b/calico/felix/test/test_fiptables.py index 9fb3916acc..ad4cb961dd 100644 --- a/calico/felix/test/test_fiptables.py +++ b/calico/felix/test/test_fiptables.py @@ -31,6 +31,7 @@ _log = logging.getLogger(__name__) +patch.object = getattr(patch, "object") # Keep PyCharm linter happy. EXTRACT_UNREF_TESTS = [ ("""Chain INPUT (policy DROP) diff --git a/calico/felix/test/test_ipsets.py b/calico/felix/test/test_ipsets.py index f6f6ea2fd1..ee957370d9 100644 --- a/calico/felix/test/test_ipsets.py +++ b/calico/felix/test/test_ipsets.py @@ -23,10 +23,13 @@ import logging from pprint import pformat from mock import * +from netaddr import IPAddress + from calico.datamodel_v1 import EndpointId -from calico.felix.futils import IPV4, FailedSystemCall -from calico.felix.ipsets import (EndpointData, IpsetManager, IpsetActor, - TagIpset, EMPTY_ENDPOINT_DATA, Ipset) +from calico.felix.futils import IPV4, FailedSystemCall, CommandOutput +from calico.felix.ipsets import (EndpointData, IpsetManager, IpsetActor, + TagIpset, EMPTY_ENDPOINT_DATA, Ipset, + list_ipset_names) from calico.felix.refcount import CREATED from calico.felix.test.base import BaseTestCase @@ -34,6 +37,7 @@ # Logger _log = logging.getLogger(__name__) +patch.object = getattr(patch, "object") # Keep PyCharm linter happy. EP_ID_1_1 = EndpointId("host1", "orch", "wl1_1", "ep1_1") EP_1_1 = { @@ -64,6 +68,26 @@ } EP_DATA_2_1 = EndpointData(["prof1"], ["10.0.0.1"]) +IPSET_LIST_OUTPUT = """Name: felix-v4-calico_net +Type: hash:ip +Revision: 2 +Header: family inet hashsize 1024 maxelem 1048576 +Size in memory: 16728 +References: 1 +Members: +10.1.0.28 +10.1.0.29 +10.1.0.19 + +Name: felix-v6-calico_net +Type: hash:ip +Revision: 2 +Header: family inet6 hashsize 1024 maxelem 1048576 +Size in memory: 16504 +References: 1 +Members: +""" + class TestIpsetManager(BaseTestCase): def setUp(self): @@ -106,6 +130,17 @@ def test_create(self): 'inet', 'hash:ip', max_elem=1234) + def test_maybe_start_gates_on_in_sync(self): + with patch("calico.felix.refcount.ReferenceManager." + "_maybe_start") as m_maybe_start: + self.mgr._maybe_start("tag-123") + self.assertFalse(m_maybe_start.called) + self.mgr.on_datamodel_in_sync(async=True) + self.step_mgr() + self.mgr._maybe_start("tag-123") + self.assertEqual(m_maybe_start.mock_calls, + [call("tag-123")]) + def test_tag_then_endpoint(self): # Send in the messages. self.mgr.on_tags_update("prof1", ["tag1"], async=True) @@ -113,6 +148,11 @@ def test_tag_then_endpoint(self): # Let the actor process them. self.step_mgr() self.assert_one_ep_one_tag() + # Undo our messages to check that the index is correctly updated, + self.mgr.on_tags_update("prof1", None, async=True) + self.mgr.on_endpoint_update(EP_ID_1_1, None, async=True) + self.step_mgr() + self.assert_index_empty() def test_endpoint_then_tag(self): # Send in the messages. @@ -141,6 +181,10 @@ def assert_one_ep_one_tag(self): } }) + def assert_index_empty(self): + self.assertEqual(self.mgr.endpoint_data_by_ep_id, {}) + self.assertEqual(self.mgr.ip_owners_by_tag, {}) + def test_change_ip(self): # Initial set-up. self.mgr.on_tags_update("prof1", ["tag1"], async=True) @@ -310,6 +354,8 @@ def on_ref_acquired(self, tag_id, ipset): @patch("calico.felix.ipsets.list_ipset_names", autospec=True) @patch("calico.felix.futils.check_call", autospec=True) def test_cleanup(self, m_check_call, m_list_ipsets): + # We're testing the in-sync processing + self.mgr.on_datamodel_in_sync(async=True) # Start with a couple ipsets. self.mgr.get_and_incref("foo", callback=self.on_ref_acquired, async=True) @@ -357,63 +403,20 @@ def test_cleanup(self, m_check_call, m_list_ipsets): call(["ipset", "destroy", "felix-v4-baz"]), ])) - def test_apply_snapshot_mainline(self): - self.mgr.apply_snapshot( - {"prof1": ["tag1"], "prof2": ["B"], "prof3": ["B"]}, - {EP_ID_1_1: EP_1_1, - EP_ID_2_1: EP_2_1}, - async=True, - ) - self.mgr.get_and_incref("tag1", - callback=self.on_ref_acquired, - async=True) - self.step_mgr() - self.mgr.on_object_startup_complete("tag1", - self.created_refs["tag1"][0], - async=True) - self.step_mgr() - self.mgr.apply_snapshot( - {"prof1": ["tag1", "tag2"]}, - {EP_ID_1_1: EP_1_1}, - async=True, - ) - self.step_mgr() - self.assertEqual(self.mgr.tags_by_prof_id, - {"prof1": ["tag1", "tag2"]}) - self.assertEqual(self.mgr.endpoint_data_by_ep_id, - {EP_ID_1_1: EP_DATA_1_1}) - ipset = self.acquired_refs["tag1"] - self.assertEqual( - ipset.replace_members.mock_calls, - [ - call(set(['10.0.0.1']), force_reprogram=True, async=True), - call(set(['10.0.0.1']), force_reprogram=True, async=True), - ] - ) - - def test_apply_snapshot_forces_reprogram(self): - # Apply a snapshot but mock the finish call so that we can check that - # apply_snapshot set the flag... - self.mgr.apply_snapshot( - {"prof1": ["A"], "prof2": ["B"]}, - {EP_ID_1_1: EP_1_1, - EP_ID_2_1: EP_2_1}, - async=True, - ) - # noinspection PyUnresolvedReferences - with patch.object(self.mgr, "_finish_msg_batch"): - self.step_actor(self.mgr) - self.assertTrue(self.mgr._force_reprogram) - - def test_finish_msg_batch_clears_reprogram_flag(self): - # Apply a snapshot and step the actor for real, should clear the flag. - self.mgr.apply_snapshot( - {"prof1": ["A"]}, - {EP_ID_1_1: EP_1_1}, - async=True, - ) + def test_update_dirty(self): + self.mgr.on_datamodel_in_sync(async=True) self.step_mgr() - self.assertFalse(self.mgr._force_reprogram) + self.mgr._dirty_tags.add("tag-123") + m_ipset = Mock(spec=TagIpset) + self.mgr.objects_by_id["tag-123"] = m_ipset + with patch.object(self.mgr, "_is_starting_or_live", + autospec=True) as m_sol: + m_sol.return_value = True + self.mgr._update_dirty_active_ipsets() + self.assertEqual( + m_ipset.replace_members.mock_calls, + [call(frozenset(), async=True)] + ) def _notify_ready(self, tags): for tag in tags: @@ -461,6 +464,7 @@ def setUp(self): self.ipset = Mock(spec=Ipset) self.ipset.max_elem = 1234 self.ipset.set_name = "felix-a_set_name" + self.ipset.temp_set_name = "felix-a_set_name-tmp" self.actor = IpsetActor(self.ipset) def test_sync_to_ipset(self): @@ -525,11 +529,44 @@ def test_sync_to_ipset(self): self.ipset.reset_mock() def test_members_too_big(self): - self.actor.members = ["1.2.3.4"] * 2000 - self.actor._sync_to_ipset() + members = set([str(IPAddress(x)) for x in range(2000)]) + self.actor.replace_members(members, async=True) + self.step_actor(self.actor) # Check we return early without updating programmed_members. self.assertEqual(self.actor.programmed_members, None) + def test_owned_ipset_names(self): + self.assertEqual(self.actor.owned_ipset_names(), + set(["felix-a_set_name", "felix-a_set_name-tmp"])) + + +class TestTagIpsetActor(BaseTestCase): + def setUp(self): + super(TestTagIpsetActor, self).setUp() + self.m_ipset = Mock(spec=Ipset) + self.m_ipset.max_elem = 1234 + self.m_ipset.set_name = "felix-a_set_name" + self.m_ipset.temp_set_name = "felix-a_set_name-tmp" + self.tag_ipset = TagIpset("tag-123", "IPv4", max_elem=1024) + self.tag_ipset._ipset = self.m_ipset + self.m_mgr = Mock() + self.tag_ipset._manager = self.m_mgr + self.tag_ipset._id = "tag-123" + + def test_lifecycle(self): + self.tag_ipset.replace_members(set(["1.2.3.4"]), async=True) + self.step_actor(self.tag_ipset) + self.assertEqual( + self.m_mgr.on_object_startup_complete.mock_calls, + [call("tag-123", self.tag_ipset, async=True)] + ) + self.tag_ipset.on_unreferenced(async=True) + self.step_actor(self.tag_ipset) + self.assertEqual( + self.m_mgr.on_object_cleanup_complete.mock_calls, + [call("tag-123", self.tag_ipset, async=True)] + ) + class TestIpset(BaseTestCase): def setUp(self): @@ -651,3 +688,9 @@ def test_delete(self, m_call_silent): call(["ipset", "destroy", "foo-tmp"]), ] ) + + @patch("calico.felix.futils.check_call", autospec=True) + def test_list_ipset_names(self, m_check_call): + m_check_call.return_value = CommandOutput(IPSET_LIST_OUTPUT, "") + self.assertEqual(list_ipset_names(), + ['felix-v4-calico_net', 'felix-v6-calico_net']) diff --git a/calico/felix/test/test_profilerules.py b/calico/felix/test/test_profilerules.py index f477726ae9..8a680e24b0 100644 --- a/calico/felix/test/test_profilerules.py +++ b/calico/felix/test/test_profilerules.py @@ -20,7 +20,8 @@ """ import logging -from mock import Mock, call +from mock import Mock, call, patch +from calico.felix import refcount from calico.felix.fiptables import IptablesUpdater from calico.felix.futils import FailedSystemCall from calico.felix.ipsets import IpsetManager, TagIpset @@ -96,6 +97,116 @@ } +class TestRulesManager(BaseTestCase): + def setUp(self): + super(TestRulesManager, self).setUp() + self.m_updater = Mock(spec=IptablesUpdater) + self.m_ipset_mgr = Mock(spec=IpsetManager) + self.mgr = RulesManager(4, self.m_updater, self.m_ipset_mgr) + + def test_create(self): + pr = self.mgr._create("profile-id") + self.assertEqual(pr.id, "profile-id") + self.assertEqual(pr.ip_version, 4) + self.assertEqual(pr._iptables_updater, self.m_updater) + self.assertEqual(pr._ipset_mgr, self.m_ipset_mgr) + + def test_on_object_started_unknown(self): + m_pr = Mock(spec=ProfileRules) + self.mgr._on_object_started("profile-id", m_pr) + self.assertEqual( + m_pr.on_profile_update.mock_calls, + [call(None, async=True)] + ) + + def test_on_object_started(self): + m_pr = Mock(spec=ProfileRules) + self.mgr.rules_by_profile_id["profile-id"] = {"foo": "bar"} + self.mgr._on_object_started("profile-id", m_pr) + self.assertEqual( + m_pr.on_profile_update.mock_calls, + [call({"foo": "bar"}, async=True)] + ) + + def test_on_datamodel_in_sync(self): + with patch("calico.felix.refcount.ReferenceManager." + "_maybe_start_all", autospec=True) as m_start: + self.mgr.on_datamodel_in_sync(async=True) + self.mgr.on_datamodel_in_sync(async=True) + self.step_actor(self.mgr) + # Only the first datamodel_in_sync triggers maybe_start_all. + self.assertEqual(m_start.mock_calls, [call(self.mgr)]) + + def test_maybe_start_known_in_sync(self): + with patch("calico.felix.refcount." + "ReferenceManager._maybe_start") as m_maybe_start: + self.mgr._maybe_start("profile-id", in_sync=True) + self.assertEqual( + m_maybe_start.mock_calls, + [call("profile-id")] + ) + + def test_maybe_start_globally_in_sync(self): + with patch("calico.felix.refcount." + "ReferenceManager._maybe_start") as m_maybe_start: + self.mgr.on_datamodel_in_sync(async=True) + self.step_actor(self.mgr) + self.mgr._maybe_start("profile-id") + self.assertEqual( + m_maybe_start.mock_calls, + [call("profile-id")] + ) + + def test_maybe_start_not_in_sync(self): + with patch("calico.felix.refcount." + "ReferenceManager._maybe_start") as m_maybe_start: + self.mgr._maybe_start("profile-id", in_sync=False) + self.assertEqual(m_maybe_start.mock_calls, []) + + def test_on_rules_update_unknown(self): + with patch("calico.felix.refcount." + "ReferenceManager._maybe_start") as m_maybe_start: + self.mgr.on_rules_update("prof-id", {"foo": "bar"}, async=True) + self.step_actor(self.mgr) + # Nothing to try to start. + self.assertEqual(m_maybe_start.mock_calls, []) + + def test_on_rules_update_not_started(self): + with patch("calico.felix.refcount." + "ReferenceManager._maybe_start") as m_maybe_start: + self.mgr.on_rules_update("prof-id", {"foo": "bar"}, async=True) + self.mgr.objects_by_id["prof-id"] = Mock() + self.step_actor(self.mgr) + # Should try to start the ProfileRules. + self.assertEqual(m_maybe_start.mock_calls, + [call("prof-id")]) + + def test_on_rules_update_started(self): + with patch("calico.felix.refcount." + "ReferenceManager._maybe_start") as m_maybe_start: + p = {"foo": "bar"} + self.mgr.on_rules_update("prof-id", p, async=True) + m_pr = Mock() + m_pr.ref_mgmt_state = refcount.LIVE + self.mgr.objects_by_id["prof-id"] = m_pr + self.step_actor(self.mgr) + self.assertEqual(m_pr.on_profile_update.mock_calls, + [call(p, force_reprogram=False, async=True)]) + # Already started so shouldn't try to start it. + self.assertEqual(m_maybe_start.mock_calls, []) + + def test_on_rules_delete(self): + with patch("calico.felix.refcount." + "ReferenceManager._maybe_start") as m_maybe_start: + self.mgr.on_rules_update("prof-id", None, async=True) + self.mgr.objects_by_id["prof-id"] = Mock() + self.step_actor(self.mgr) + # Even though we know it's gone, still try to start it. If it's + # referenced this will ensure that the chain is cleaned up. + self.assertEqual(m_maybe_start.mock_calls, + [call("prof-id")]) + + class TestProfileRules(BaseTestCase): def setUp(self): super(TestProfileRules, self).setUp() @@ -254,6 +365,11 @@ def test_early_unreferenced(self): self.m_ipt_updater.delete_chains.assert_called_once_with( set(['felix-p-prof1-i', 'felix-p-prof1-o']), async=False ) + # Further calls should be ignored + self.m_ipt_updater.reset_mock() + self.rules.on_unreferenced(async=True) + self.step_actor(self.rules) + self.assertFalse(self.m_ipt_updater.delete_chains.called) def test_unreferenced_after_creation(self): """ diff --git a/calico/felix/test/test_splitter.py b/calico/felix/test/test_splitter.py index dfb9b7c39a..2b1c73e812 100644 --- a/calico/felix/test/test_splitter.py +++ b/calico/felix/test/test_splitter.py @@ -31,6 +31,7 @@ # A mocked config object for use in the UpdateSplitter. Config = collections.namedtuple('Config', ['STARTUP_CLEANUP_DELAY']) + class TestUpdateSplitter(BaseTestCase): """ Tests for the UpdateSplitter actor. @@ -56,117 +57,18 @@ def get_splitter(self): self.masq_manager ) - def test_apply_whole_snapshot_clean(self): - """ - Test that a whole snapshot applies cleanly to all managers. - """ - # We apply a simple sentinel map. The exact map we use really shouldn't - # matter here. We do, however, use different ones for rules, tags, and - # endpoints. - rules = {'profileA': ['first rule', 'second rule']} - tags = {'profileA': ['first tag', 'second tag']} - endpoints = {'endpointA': 'endpoint object'} - ipv4_pools_by_id = {"10.0.0.1-5": {"cidr": "10.0.0.1/5", - "masquerade": True}} - s = self.get_splitter() - - # Apply the snapshot and let it run. - s.apply_snapshot(rules, tags, endpoints, ipv4_pools_by_id, async=True) - self.step_actor(s) - - # At this point, each of our managers should have been notified (one - # call to apply_snapshot), but cleanup should not have occurred. - for mgr in self.ipsets_mgrs: - mgr.apply_snapshot.assertCalledOnceWith( - tags, endpoints, async=True - ) - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.rules_mgrs: - mgr.apply_snapshot.assertCalledOnceWith(rules, async=True) - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.endpoint_mgrs: - mgr.apply_snapshot.assertCalledOnceWith(endpoints, async=True) - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.iptables_updaters: - self.assertEqual(mgr.cleanup.call_count, 0) - self.masq_manager.apply_snapshot.assert_called_once_with( - ipv4_pools_by_id, async=True) - - # If we spin the scheduler again, we should begin cleanup. - # Warning: this might be a bit brittle, we may not be waiting long - # enough here, at least on busy machines. - gevent.sleep(0.1) - self.step_actor(s) - - # Confirm that we cleaned up. Cleanup only affects the - # iptables_updaters and the ipsets_managers, so confirm the other - # managers got left alone. - for mgr in self.ipsets_mgrs: - mgr.cleanup.assertCalledOnceWith(async=False) - for mgr in self.rules_mgrs: - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.endpoint_mgrs: - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.iptables_updaters: - mgr.cleanup.assertCalledOnceWith(async=False) - - def test_repeated_snapshots_clean_up_only_once(self): - """ - Test that repeated snapshots only clean up once. - """ - # We apply a simple sentinel map. The exact map we use really shouldn't - # matter here. We do, however, use different ones for rules, tags, and - # endpoints. - rules = {'profileA': ['first rule', 'second rule']} - tags = {'profileA': ['first tag', 'second tag']} - endpoints = {'endpointA': 'endpoint object'} - ipv4_pools_by_id = {} + def test_on_datamodel_in_sync(self): s = self.get_splitter() - - # Apply three snapshots and let them run. Because of batching logic, - # we should only need to spin the actor once. - s.apply_snapshot(rules, tags, endpoints, ipv4_pools_by_id, async=True) - s.apply_snapshot(rules, tags, endpoints, ipv4_pools_by_id, async=True) - s.apply_snapshot(rules, tags, endpoints, ipv4_pools_by_id, async=True) - self.step_actor(s) - - # At this point, each of our managers should have been notified (one - # call to apply_snapshot), but cleanup should not have occurred. - for mgr in self.ipsets_mgrs: - mgr.apply_snapshot.assertCalledWith( - tags, endpoints, async=True - ) - self.assertEqual(mgr.apply_snapshot.call_count, 3) - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.rules_mgrs: - mgr.apply_snapshot.assertCalledWith(rules, async=True) - self.assertEqual(mgr.apply_snapshot.call_count, 3) - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.endpoint_mgrs: - mgr.apply_snapshot.assertCalledWith(endpoints, async=True) - self.assertEqual(mgr.apply_snapshot.call_count, 3) - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.iptables_updaters: - self.assertEqual(mgr.cleanup.call_count, 0) - self.assertEqual(self.masq_manager.apply_snapshot.call_count, 3) - - # If we spin the scheduler again, we should begin cleanup. - # Warning: this might be a bit brittle, we may not be waiting long - # enough here, at least on busy machines. - gevent.sleep(0.1) - self.step_actor(s) - - # Confirm that we cleaned up. Cleanup only affects the - # iptables_updaters and the ipsets_managagers, so confirm the other - # managers got left alone. - for mgr in self.ipsets_mgrs: - mgr.cleanup.assertCalledOnceWith(async=False) - for mgr in self.rules_mgrs: - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.endpoint_mgrs: - self.assertEqual(mgr.cleanup.call_count, 0) - for mgr in self.iptables_updaters: - mgr.cleanup.assertCalledOnceWith(async=False) + with mock.patch("gevent.spawn_later") as m_spawn: + s.on_datamodel_in_sync(async=True) + s.on_datamodel_in_sync(async=True) + self.step_actor(s) + self.assertTrue(s._cleanup_scheduled) + self.assertEqual(m_spawn.mock_calls, + [mock.call(0, mock.ANY)]) + for mgr in self.ipsets_mgrs + self.rules_mgrs + self.endpoint_mgrs: + self.assertEqual(mgr.on_datamodel_in_sync.mock_calls, + [mock.call(async=True), mock.call(async=True)]) def test_cleanup_give_up_on_exception(self): """ @@ -178,13 +80,20 @@ def test_cleanup_give_up_on_exception(self): # However, make sure that the first ipset manager and the first # iptables updater throw exceptions when called. self.ipsets_mgrs[0].cleanup.side_effect = RuntimeError('Bang!') - self.iptables_updaters[0].cleanup.side_effect = RuntimeError('Bang!') # Start the cleanup. result = s.trigger_cleanup(async=True) self.step_actor(s) self.assertRaises(RuntimeError, result.get) + def test_cleanup_mainline(self): + # No need to apply any data here. + s = self.get_splitter() + # Start the cleanup. + result = s.trigger_cleanup(async=True) + self.step_actor(s) + result.get() + def test_rule_updates_propagate(self): """ Test that the on_rules_update message propagates correctly. diff --git a/calico/geventutils.py b/calico/geventutils.py new file mode 100644 index 0000000000..a2bb301cd6 --- /dev/null +++ b/calico/geventutils.py @@ -0,0 +1,53 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +calico.geventutils +~~~~~~~~~~~~~~~~~~ + +Helper utilities for gevent. +""" +import itertools +import logging + +import gevent +import gevent.local + +_log = logging.getLogger(__name__) + + +tid_storage = gevent.local.local() +tid_counter = itertools.count() +# Ought to do itertools.count(start=1), but python 2.6 does not support it. +tid_counter.next() + + +def greenlet_id(): + """ + Returns an integer greenlet ID. + itertools.count() is atomic, if the internet is correct. + http://stackoverflow.com/questions/23547604/python-counter-atomic-increment + """ + try: + tid = tid_storage.tid + except: + tid = tid_counter.next() + tid_storage.tid = tid + return tid + + +class GreenletFilter(logging.Filter): + def filter(self, record): + record.tid = greenlet_id() + return True \ No newline at end of file diff --git a/calico/test/lib.py b/calico/test/lib.py index aa079ac4bf..31cde72c77 100644 --- a/calico/test/lib.py +++ b/calico/test/lib.py @@ -87,7 +87,7 @@ class EtcdException(Exception): pass -class EtcdKeyNotFound(EtcdException): +class EtcdValueError(EtcdException, ValueError): pass @@ -95,11 +95,23 @@ class EtcdClusterIdChanged(EtcdException): pass +class EtcdKeyError(EtcdException): + pass + + +class EtcdKeyNotFound(EtcdKeyError): + pass + + class EtcdEventIndexCleared(EtcdException): pass -class EtcdValueError(EtcdException): +class EtcdConnectionFailed(EtcdException): + pass + + +class EtcdWatcherCleared(EtcdException): pass @@ -110,6 +122,7 @@ class EtcdDirNotEmpty(EtcdValueError): m_etcd.EtcdException = EtcdException m_etcd.EtcdKeyNotFound = EtcdKeyNotFound m_etcd.EtcdClusterIdChanged = EtcdClusterIdChanged +m_etcd.EtcdConnectionFailed = EtcdConnectionFailed m_etcd.EtcdEventIndexCleared = EtcdEventIndexCleared m_etcd.EtcdValueError = EtcdValueError m_etcd.EtcdDirNotEmpty = EtcdDirNotEmpty diff --git a/calico/test/test_common.py b/calico/test/test_common.py index ebcf59e993..901bdbb4f2 100644 --- a/calico/test/test_common.py +++ b/calico/test/test_common.py @@ -686,19 +686,6 @@ def test_validate_tags(self): "Invalid tag"): common.validate_tags(profile_id, ["value", "bad value"]) - def test_greenlet_id(self): - def greenlet_run(): - tid = common.greenlet_id() - return tid - - tid = common.greenlet_id() - child = eventlet.spawn(greenlet_run) - child_tid = child.wait() - new_tid = common.greenlet_id() - - self.assertTrue(child_tid > tid) - self.assertEqual(tid, new_tid) - def test_validate_ipam_pool(self): self.assert_ipam_pool_valid({"cidr": "10/16", "foo": "bar"}, {"cidr": "10.0.0.0/16"}, 4) diff --git a/calico/test/test_etcdutils.py b/calico/test/test_etcdutils.py index 93bfe69296..01354a9cf9 100644 --- a/calico/test/test_etcdutils.py +++ b/calico/test/test_etcdutils.py @@ -21,14 +21,24 @@ import logging import types -import etcd + +from etcd import EtcdException from mock import Mock, patch, call -from calico.etcdutils import PathDispatcher, EtcdWatcher, delete_empty_parents +from urllib3.exceptions import ReadTimeoutError + +from calico.etcdutils import ( + PathDispatcher, EtcdWatcher, delete_empty_parents, + EtcdClientOwner, ResyncRequired +) +# Since other tests patch the module table, make sure we have the same etcd +# module as the module under test. +from calico.etcdutils import etcd -from calico.felix.test.base import BaseTestCase +from calico.felix.test.base import BaseTestCase, ExpectedException _log = logging.getLogger(__name__) +patch.object = getattr(patch, "object") # Keep PyCharm linter happy. SAME_AS_KEY = object() @@ -93,6 +103,11 @@ def test_delete_empty_parents_other_exception(self): ] ) + def test_delete_empty_parents_bad_prefix(self): + self.assertRaises(ValueError, + delete_empty_parents, + Mock(), "/foo/bar/baz/biff", "/bar") + class _TestPathDispatcherBase(BaseTestCase): """ @@ -228,13 +243,93 @@ class TestDispatcherExpire(_TestPathDispatcherBase): expected_handlers = "delete" +class TestEtcdClientOwner(BaseTestCase): + @patch("etcd.Client", autospec=True) + def test_create(self, m_client_cls): + owner = EtcdClientOwner("localhost:1234") + m_client = m_client_cls.return_value + m_client.expected_cluster_id = "abcdef" + owner.reconnect() + self.assertEqual(m_client_cls.mock_calls, + [call(host="localhost", port=1234, + expected_cluster_id=None), + call().__nonzero__(), + call(host="localhost", port=1234, + expected_cluster_id="abcdef"),]) + + @patch("etcd.Client", autospec=True) + def test_create_default(self, m_client): + owner = EtcdClientOwner("localhost") + self.assertEqual(m_client.mock_calls, + [call(host="localhost", port=4001, + expected_cluster_id=None)]) + + class TestEtcdWatcher(BaseTestCase): def setUp(self): super(TestEtcdWatcher, self).setUp() - with patch("calico.etcdutils.EtcdWatcher.reconnect") as m_reconnect: - self.watcher = EtcdWatcher("foobar:4001", "/calico") + self.reconnect_patch = patch("calico.etcdutils.EtcdWatcher.reconnect") + self.m_reconnect = self.reconnect_patch.start() + self.watcher = EtcdWatcher("foobar:4001", "/calico") self.m_client = Mock() self.watcher.client = self.m_client + self.m_dispatcher = Mock(spec=PathDispatcher) + self.watcher.dispatcher = self.m_dispatcher + + @patch("time.sleep", autospec=True) + def test_mainline(self, m_sleep): + m_snap_response = Mock() + m_snap_response.etcd_index = 1 + m_poll_response = Mock() + m_poll_response.modifiedIndex = 2 + responses = [ + m_snap_response, m_poll_response, ResyncRequired(), # Loop 1 + EtcdException(), # Loop 2 + ExpectedException(), # Loop 3, Break out of loop. + ] + self.m_client.read.side_effect = iter(responses) + with patch.object(self.watcher, "_on_pre_resync", + autospec=True) as m_pre_r: + with patch.object(self.watcher, "_on_snapshot_loaded", + autospec=True) as m_snap_load: + self.assertRaises(ExpectedException, self.watcher.loop) + # _on_pre_resync() called once per loop. + self.assertEqual(m_pre_r.mock_calls, [call(), call(), call()]) + # The snapshot only loads successfully the first time. + self.assertEqual(m_snap_load.mock_calls, [call(m_snap_response)]) + self.assertEqual(self.m_dispatcher.handle_event.mock_calls, + [call(m_poll_response)]) + # Should sleep after exception. + m_sleep.assert_called_once_with(1) + + def test_loop_stopped(self): + self.watcher._stopped = True + + with patch.object(self.watcher, "_on_pre_resync", + autospec=True) as m_pre_r: + self.watcher.loop() + self.assertFalse(m_pre_r.called) + + def test_register(self): + self.watcher.register_path("key", foo="bar") + self.assertEqual(self.m_dispatcher.register.mock_calls, + [call("key", foo="bar")]) + + @patch("time.sleep", autospec=True) + def test_wait_for_ready(self, m_sleep): + m_resp_1 = Mock() + m_resp_1.value = "false" + m_resp_2 = Mock() + m_resp_2.value = "true" + responses = [ + etcd.EtcdException(), + etcd.EtcdKeyNotFound(), + m_resp_1, + m_resp_2, + ] + self.m_client.read.side_effect = iter(responses) + self.watcher.wait_for_ready(1) + self.assertEqual(m_sleep.mock_calls, [call(1)] * 3) def test_load_initial_dump(self): m_response = Mock(spec=etcd.EtcdResult) @@ -252,3 +347,68 @@ def test_load_initial_dump(self): call("/calico", recursive=True), ]) self.assertEqual(self.watcher.next_etcd_index, 10001) + + def test_load_initial_dump_stopped(self): + self.watcher.stop() + self.m_client.read.side_effect = etcd.EtcdKeyNotFound() + self.assertRaises(etcd.EtcdKeyNotFound, self.watcher.load_initial_dump) + + def test_resync_set(self): + self.watcher.next_etcd_index = 1 + self.watcher.resync_after_current_poll = True + self.assertRaises(ResyncRequired, self.watcher.wait_for_etcd_event) + self.assertFalse(self.watcher.resync_after_current_poll) + + @patch("time.sleep", autospec=True) + def test_wait_for_etcd_event_conn_failed(self, m_sleep): + self.watcher.next_etcd_index = 1 + m_resp = Mock() + m_resp.modifiedIndex = 123 + read_timeout = etcd.EtcdConnectionFailed() + read_timeout.cause = ReadTimeoutError(Mock(), "", "") + other_error = etcd.EtcdConnectionFailed() + other_error.cause = ExpectedException() + responses = [ + read_timeout, + other_error, + m_resp, + ] + self.m_client.read.side_effect = iter(responses) + event = self.watcher.wait_for_etcd_event() + self.assertEqual(event, m_resp) + self.assertEqual(m_sleep.mock_calls, [call(1)]) + + def test_wait_for_etcd_event_cluster_id_changed(self): + self.watcher.next_etcd_index = 1 + responses = [ + etcd.EtcdClusterIdChanged(), + ] + self.m_client.read.side_effect = iter(responses) + self.assertRaises(ResyncRequired, self.watcher.wait_for_etcd_event) + + def test_wait_for_etcd_event_index_cleared(self): + self.watcher.next_etcd_index = 1 + responses = [ + etcd.EtcdEventIndexCleared(), + ] + self.m_client.read.side_effect = iter(responses) + self.assertRaises(ResyncRequired, self.watcher.wait_for_etcd_event) + + @patch("time.sleep", autospec=True) + def test_wait_for_etcd_event_unexpected_error(self, m_sleep): + self.watcher.next_etcd_index = 1 + responses = [ + etcd.EtcdException(), + ] + self.m_client.read.side_effect = iter(responses) + self.assertRaises(ResyncRequired, self.watcher.wait_for_etcd_event) + self.assertEqual(m_sleep.mock_calls, [call(1)]) + + def test_coverage(self): + # These methods are no-ops. + self.watcher._on_pre_resync() + self.watcher._on_snapshot_loaded(Mock()) + + def tearDown(self): + self.reconnect_patch.stop() + super(TestEtcdWatcher, self).tearDown() diff --git a/calico/test/test_geventutils.py b/calico/test/test_geventutils.py new file mode 100644 index 0000000000..03ab3210b2 --- /dev/null +++ b/calico/test/test_geventutils.py @@ -0,0 +1,44 @@ +# -*- coding: utf-8 -*- +# Copyright 2015 Metaswitch Networks +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +test_geventutils +~~~~~~~~~~~~~~~~ + +Test code for gevent utility functions. +""" + +import logging +import gevent +from calico import geventutils + +from calico.felix.test.base import BaseTestCase + +_log = logging.getLogger(__name__) + + +class TestGreenletUtils(BaseTestCase): + + def test_greenlet_id(self): + def greenlet_run(): + tid = geventutils.greenlet_id() + return tid + + tid = geventutils.greenlet_id() + child = gevent.spawn(greenlet_run) + child_tid = child.get() + new_tid = geventutils.greenlet_id() + + self.assertTrue(child_tid > tid) + self.assertEqual(tid, new_tid) diff --git a/debian/calico-felix.logrotate b/debian/calico-felix.logrotate index 876a3b57c3..15f7093628 100644 --- a/debian/calico-felix.logrotate +++ b/debian/calico-felix.logrotate @@ -5,3 +5,10 @@ delaycompress minsize 1M } +/var/log/calico/felix-etcd.log { + daily + missingok + compress + delaycompress + minsize 1M +} diff --git a/debian/changelog b/debian/changelog index 7b0880a2ef..ec6805db74 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,10 @@ +calico (1.3.0~pre.1) trusty; urgency=medium + + * Felix now parses the etcd snapshot in parallel with the event stream; + this dramatically increases scale when under load. + + -- Shaun Crampton Mon, 26 Oct 2015 13:41:00 +0100 + calico (1.2.0-1) trusty; urgency=medium * Truncate long output from FailedSystemCall exception. diff --git a/debian/control b/debian/control index 49db4c2863..b1536919f7 100644 --- a/debian/control +++ b/debian/control @@ -52,7 +52,12 @@ Depends: ${misc:Depends}, ${python:Depends}, ${shlibs:Depends}, - python-etcd (>= 0.4.1+calico.1) + python-etcd (>= 0.4.1+calico.1), + python-ijson (>= 2.2-1), + python-datrie (>= 0.7-1), + libyajl2 (>= 2.0.4-4), + libdatrie1 (>= 0.2.8-1), + python-msgpack (>= 0.3.0-1ubuntu3) Description: Project Calico virtual networking for cloud data centers. Project Calico is an open source solution for virtual networking in cloud data centers. Its IP-centric architecture offers numerous diff --git a/docs/source/configuration.rst b/docs/source/configuration.rst index eda3cca4d2..0b462cf4a9 100644 --- a/docs/source/configuration.rst +++ b/docs/source/configuration.rst @@ -73,60 +73,62 @@ environment variables or etcd is often more convenient. The full list of parameters which can be set is as follows. -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| Setting | Default | Meaning | -+=============================+===========================+===========================================================================================+ -| EtcdAddr | localhost:4001 | The location (IP / hostname and port) of the etcd node or proxy that Felix should connect | -| | | to. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| DefaultEndpointToHostAction | DROP | By default Calico blocks traffic from endpoints to the host itself by using an iptables | -| | | DROP action. If you want to allow some or all traffic from endpoint to host then set | -| | | this parameter to "RETURN" (which causes the rest of the iptables INPUT chain to be | -| | | processed) or "ACCEPT" (which immediately accepts packets). | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| FelixHostname | socket.gethostname() | The hostname Felix reports to the plugin. Should be used if the hostname Felix | -| | | autodetects is incorrect or does not match what the plugin will expect. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| MetadataAddr | 127.0.0.1 | The IP address or domain name of the server that can answer VM queries for cloud-init | -| | | metadata. In OpenStack, this corresponds to the machine running nova-api (or in Ubuntu, | -| | | nova-api-metadata). A value of 'None' (case insensitive) means that Felix should not set | -| | | up any NAT rule for the metadata path. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| MetadataPort | 8775 | The port of the metadata server. This, combined with global.MetadataAddr (if not 'None'), | -| | | is used to set up a NAT rule, from 169.254.169.254:80 to MetadataAddr:MetadataPort. In | -| | | most cases this should not need to be changed. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| InterfacePrefix | None | The start of the interface name for all interfaces. This is set to "tap" on OpenStack | -| | | by the plugin, but must be set to "veth" on most Docker deployments. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| LogFilePath | /var/log/calico/felix.log | The full path to the felix log. Set to "none" to disable file logging. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| LogSeveritySys | ERROR | The log severity above which logs are sent to the syslog. Valid values are DEBUG, INFO, | -| | | WARNING, ERROR and CRITICAL, or NONE for no logging to syslog (all values case | -| | | insensitive). | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| LogSeverityFile | INFO | The log severity above which logs are sent to the log file. Valid values as for | -| | | LogSeveritySys. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| LogSeverityScreen | ERROR | The log severity above which logs are sent to the stdout. Valid values as for | -| | | LogSeveritySys. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| StartupCleanupDelay | 30 | Delay, in seconds, before felix does its start-of-day cleanup to remove orphaned iptables | -| | | chains and ipsets. Before the first cleanup, felix operates in "graceful restart" mode, | -| | | during which it preserves any pre-existing chains and ipsets. | -| | | | -| | | In a large deployment you may want to increase this value to give felix more time to | -| | | load the initial snapshot from etcd before cleaning up. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| PeriodicResyncInterval | 3600 | Period, in seconds, at which felix does a full resync with etcd and reprograms | -| | | iptables/ipsets. Set to 0 to disable periodic resync. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| IptablesRefreshInterval | 60 | Period, in seconds, at which felix re-applies all iptables state to ensure that no other | -| | | process has accidentally broken Calico's rules. Set to 0 to disable iptables refresh. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ -| MaxIpsetSize | 1048576 | Maximum size for the ipsets used by Felix to implement tags. Should be set to a number | -| | | that is greater than the maximum number of IP addresses that are ever expected in a tag. | -+-----------------------------+---------------------------+-------------------------------------------------------------------------------------------+ ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| Setting | Default | Meaning | ++=============================+================================+===========================================================================================+ +| EtcdAddr | localhost:4001 | The location (IP / hostname and port) of the etcd node or proxy that Felix should connect | +| | | to. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| DefaultEndpointToHostAction | DROP | By default Calico blocks traffic from endpoints to the host itself by using an iptables | +| | | DROP action. If you want to allow some or all traffic from endpoint to host then set | +| | | this parameter to "RETURN" (which causes the rest of the iptables INPUT chain to be | +| | | processed) or "ACCEPT" (which immediately accepts packets). | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| FelixHostname | socket.gethostname() | The hostname Felix reports to the plugin. Should be used if the hostname Felix | +| | | autodetects is incorrect or does not match what the plugin will expect. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| MetadataAddr | 127.0.0.1 | The IP address or domain name of the server that can answer VM queries for cloud-init | +| | | metadata. In OpenStack, this corresponds to the machine running nova-api (or in Ubuntu, | +| | | nova-api-metadata). A value of 'None' (case insensitive) means that Felix should not set | +| | | up any NAT rule for the metadata path. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| MetadataPort | 8775 | The port of the metadata server. This, combined with global.MetadataAddr (if not 'None'), | +| | | is used to set up a NAT rule, from 169.254.169.254:80 to MetadataAddr:MetadataPort. In | +| | | most cases this should not need to be changed. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| InterfacePrefix | None | The start of the interface name for all interfaces. This is set to "tap" on OpenStack | +| | | by the plugin, but must be set to "veth" on most Docker deployments. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| LogFilePath | /var/log/calico/felix.log | The full path to the felix log. Set to "none" to disable file logging. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| EtcdDriverLogFilePath | /var/log/calico/felix-etcd.log | Felix's etcd driver has its own log file. This parameter contains its full path. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| LogSeveritySys | ERROR | The log severity above which logs are sent to the syslog. Valid values are DEBUG, INFO, | +| | | WARNING, ERROR and CRITICAL, or NONE for no logging to syslog (all values case | +| | | insensitive). | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| LogSeverityFile | INFO | The log severity above which logs are sent to the log file. Valid values as for | +| | | LogSeveritySys. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| LogSeverityScreen | ERROR | The log severity above which logs are sent to the stdout. Valid values as for | +| | | LogSeveritySys. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| StartupCleanupDelay | 30 | Delay, in seconds, before felix does its start-of-day cleanup to remove orphaned iptables | +| | | chains and ipsets. Before the first cleanup, felix operates in "graceful restart" mode, | +| | | during which it preserves any pre-existing chains and ipsets. | +| | | | +| | | In a large deployment you may want to increase this value to give felix more time to | +| | | load the initial snapshot from etcd before cleaning up. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| PeriodicResyncInterval | 3600 | Period, in seconds, at which felix does a full resync with etcd and reprograms | +| | | iptables/ipsets. Set to 0 to disable periodic resync. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| IptablesRefreshInterval | 60 | Period, in seconds, at which felix re-applies all iptables state to ensure that no other | +| | | process has accidentally broken Calico's rules. Set to 0 to disable iptables refresh. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ +| MaxIpsetSize | 1048576 | Maximum size for the ipsets used by Felix to implement tags. Should be set to a number | +| | | that is greater than the maximum number of IP addresses that are ever expected in a tag. | ++-----------------------------+--------------------------------+-------------------------------------------------------------------------------------------+ Environment variables diff --git a/felix_requirements.txt b/felix_requirements.txt index 12ebfaf463..00019c577d 100644 --- a/felix_requirements.txt +++ b/felix_requirements.txt @@ -3,3 +3,7 @@ greenlet netaddr python-etcd>=0.4.1 posix-spawn>=0.2.post6 +datrie>=0.7 +ijson>=2.2 +msgpack-python>=0.3 +urllib3>=1.7.1 diff --git a/rpm/calico-felix.logrotate b/rpm/calico-felix.logrotate index 876a3b57c3..15f7093628 100644 --- a/rpm/calico-felix.logrotate +++ b/rpm/calico-felix.logrotate @@ -5,3 +5,10 @@ delaycompress minsize 1M } +/var/log/calico/felix-etcd.log { + daily + missingok + compress + delaycompress + minsize 1M +} diff --git a/rpm/calico.spec b/rpm/calico.spec index e7a5b04417..293923b2c9 100644 --- a/rpm/calico.spec +++ b/rpm/calico.spec @@ -93,7 +93,7 @@ This package provides common files. %package felix Group: Applications/Engineering Summary: Project Calico virtual networking for cloud data centers -Requires: calico-common, conntrack-tools, ipset, iptables, net-tools, python-devel, python-netaddr, python-gevent +Requires: calico-common, conntrack-tools, ipset, iptables, net-tools, python-devel, python-netaddr, python-gevent, datrie, ijson, python-urllib3, python-msgpack %description felix diff --git a/tox.ini b/tox.ini index 4b5d94a677..3ddf2bc04b 100644 --- a/tox.ini +++ b/tox.ini @@ -18,6 +18,7 @@ commands = ./tox-cover.sh thread calico.test ./tox-cover.sh gevent calico.felix ./tox-cover.sh eventlet calico.openstack + ./tox-cover.sh thread calico.etcddriver coverage report -m [testenv:pypy] @@ -27,6 +28,7 @@ commands = commands = ./tox-cover.sh thread calico.test nosetests calico.felix + ./tox-cover.sh thread calico.etcddriver coverage report -m deps = nose