aboutsummaryrefslogtreecommitdiffstats
path: root/hicn-light/src/hicn/core/mapme.c
diff options
context:
space:
mode:
Diffstat (limited to 'hicn-light/src/hicn/core/mapme.c')
-rw-r--r--hicn-light/src/hicn/core/mapme.c1600
1 files changed, 850 insertions, 750 deletions
diff --git a/hicn-light/src/hicn/core/mapme.c b/hicn-light/src/hicn/core/mapme.c
index a22d01ae7..acda2b8cd 100644
--- a/hicn-light/src/hicn/core/mapme.c
+++ b/hicn-light/src/hicn/core/mapme.c
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2017-2019 Cisco and/or its affiliates.
+ * Copyright (c) 2021-2023 Cisco and/or its affiliates.
* 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:
@@ -16,6 +16,98 @@
/**
* @file mapme.c
* @brief MAP-Me : AnchorLess Producer Mobility Management.
+ *
+ * TODO:
+ * - review notification code with to integration of VPP implementation
+ * - reflect changes back in VPP
+ * - implement heuristic for update/notification selection
+ *
+ * MAP-Me hooks in forwarder
+ *
+ * A) Face table changes
+ *
+ * - face added
+ *
+ * * new local/producer face : this is a new prefix that we need to advertise
+ * on existing connections.
+ *
+ * We go over non-local connections an advertise the prefix through an IU
+ * provided that the connection satisfies the policy associated to the FIB
+ * entry. MAP-Me assumes the prefix already exists in the network, and the
+ * IU shall be discarded if the entry does not exist at the next hop. Three
+ * possibilities:
+ * . a bootstrap mechanism
+ * . we allow subprefixes of a prefix that is not empty by duplicating the
+ * FIB entry
+ * . we allow prefix creation in all circumstances : this is problematic
+ * since we might be creating spurious entries in routers for which we
+ * don't expect entries to be created.
+ *
+ * NOTE: because in general we will not allow for FIB entry creation, we
+ * cannot let the forwarder remove FIB entries with no nexthop (for instance
+ * after the producer leaves a point-of-attachment). This might creates
+ * permanent state in router's tables, but we assume it is the role of the
+ * routing plane to take care of routing entries.
+ *
+ * * new non-local face : a new face is available (eg. thanks to the face
+ * manager, after the node has connection to a new WiFi/LTE access point),
+ * and we thus need to advertise all local/producer prefixes onto this
+ * interface.
+ *
+ * For this, we currently scan the FIB for entries that have at least one
+ * local/producer face in nexthops, advertise the prefix on this new
+ * connection provided that it satisfies the associated policy.
+ *
+ * - face removed
+ *
+ * Currently, we take no action when a face is removed. It might however be a
+ * signal that a producer application is no more running at a given node, and
+ * that we can temporarily disable the forwarding towards that path.
+ *
+ * - face up / down
+ *
+ * - face nexthop added
+ *
+ * - face changed priority/tags
+ *
+ * B) Interest and Data forwarder path
+ *
+ * mapme_on_interest
+ *
+ * mapme_on_data
+ *
+ *
+ * EVENTS
+ * NH_SET
+ * NH_ADD
+ * PH_ADD
+ * PH_DEL
+ *
+ * C) Retransmission management
+ *
+ * Data structure
+ *
+ * mapme_on_timeout
+ *
+ *
+ * This allows us to define a convenient API for implementing MAP-Me:
+ *
+ * mapme_on_face_event XXX rename
+ *
+ * mapme_set_all_adjacencies(const mapme_t *mapme, fib_entry_t *entry)
+ * This function is used to update all the adjacencies. It needs to be called
+ * in case of face add/delete/change (priority/tas) and polocy
+ *
+ * mapme_set_adjacencies(const mapme_t *mapme, fib_entry_t *entry,
+ * nexthops_t *nexthops)
+ * This function updates only the nexthops and clear the tfib. It
+ * needs to be called by the forwarding strategy in case of path switch
+ *
+ * mapme_update_adjacencies(const mapme_t *mapme, fib_entry_t *entry,
+ * bool inc_iu_seq)
+ * This function is called to propagate the IU and it propagates the IU using
+ * the nexthops in the tfib. It needs to be used for mapme prcessing, IU
+ * forwarding, NATs and in case of timeouts
*/
#ifdef WITH_MAPME
@@ -24,697 +116,779 @@
#include <hicn/core/mapme.h>
#include <stdio.h> // printf
+#include <hicn/base/loop.h>
#include <hicn/core/connection.h>
-#include <hicn/core/connectionList.h>
#include <hicn/core/forwarder.h>
-#include <hicn/core/logger.h>
-#include <hicn/core/message.h>
-#include <hicn/core/messagePacketType.h> // packet types
+#include <hicn/core/msgbuf.h>
#include <hicn/core/ticks.h>
-#include <hicn/processor/fibEntry.h>
-#include <hicn/processor/pitEntry.h>
-
-#include <parc/algol/parc_HashMap.h>
-#include <parc/algol/parc_Iterator.h>
-#include <parc/algol/parc_Unsigned.h>
-#include <parc/assert/parc_Assert.h>
+#include <hicn/core/fib_entry.h>
+#include <hicn/core/pit.h>
+#include <hicn/base/loop.h>
+#include <hicn/util/log.h>
#define MS2NS(x) x * 1000000
#define T2NS(x) forwarder_TicksToNanos(x)
#define MAPME_DEFAULT_TU 5000 /* ms */
#define MAPME_DEFAULT_RETX 500 /* ms */
-#define MAX_RETX 3
+#define MAPME_DEFAULT_DISCOVERY false
+#define MAPME_DEFAULT_PROTOCOL IPPROTO_IPV6
+#define MAPME_MAX_RETX 3
+#define MTU 1500 // XXX TODO Mutualize this define
-#define NOT_A_NOTIFICATION false
-#define NO_INGRESS 0
+#define DONT_QUEUE false
#define TIMER_NO_REPEAT false
-#define DO_DISCOVERY 1
#define MAPME_INVALID_DICOVERY_SEQ -1
+#define INIT_SEQ 0
-#define LOG_FACILITY LoggerFacility_Core
+#define foreach_mapme_event \
+ _(UNDEFINED) \
+ _(FACE_ADD) \
+ _(FACE_DEL) \
+ _(NH_SET) \
+ _(NH_ADD) \
+ _(PH_ADD) \
+ _(PH_DEL) \
+ _(N)
+
+typedef enum {
+#define _(x) MAPME_EVENT_##x,
+ foreach_mapme_event
+#undef _
+} mapme_event_t;
-#define LOG(mapme, log_level, fmt, ...) \
- do { \
- Logger *logger = forwarder_GetLogger(mapme->forwarder); \
- if (logger_IsLoggable(logger, LOG_FACILITY, log_level)) { \
- logger_Log(logger, LOG_FACILITY, log_level, __func__, fmt, \
- ##__VA_ARGS__); \
- } \
- } while (0)
+/*
+ * We need a retransmission pool holding all necessary information for crafting
+ * special interests, thus including both the DPO and the prefix associated to
+ * it.
+ */
+#define NUM_RETX_ENTRIES 100
+#define NUM_RETX_SLOT 2
-#define WARN(mapme, fmt, ...) \
- LOG(mapme, PARCLogLevel_Warning, fmt, ##__VA_ARGS__)
-#define ERR(mapme, fmt, ...) LOG(mapme, PARCLogLevel_Error, fmt, ##__VA_ARGS__)
-#define INFO(mapme, fmt, ...) LOG(mapme, PARCLogLevel_Info, fmt, ##__VA_ARGS__)
-#define DEBUG(mapme, fmt, ...) \
- LOG(mapme, PARCLogLevel_Debug, fmt, ##__VA_ARGS__)
+typedef struct {
+ hicn_prefix_t prefix;
+ fib_entry_t *entry;
+ uint8_t retx_count; // Number of retransmissions since last tfib addition
+} mapme_retx_t;
/**
* MAP-Me state data structure
*/
-struct mapme {
- uint32_t retx; /* ms */
- uint32_t Tu; /* ms */
- bool removeFibEntries;
+struct mapme_s {
+ /* Options XXX mapme_conf_t ! */
+ uint32_t retx; /* retx timeout (in ms) */
+ uint32_t timescale; /* timescale (in ms) */
+ bool discovery; /* discovery flag */
+ int protocol;
+ bool enabled; /* mapme enabled/disabled */
+
+ /*
+ * Retransmissions
+ * Lite calendar queue with NUM_RETX_SLOT slots
+ */
+ event_t *timer;
+ mapme_retx_t retx_array[NUM_RETX_SLOT][NUM_RETX_ENTRIES];
+ uint8_t retx_len[NUM_RETX_SLOT];
+ uint8_t cur;
+ uint8_t idle;
- Forwarder *forwarder;
+ forwarder_t *forwarder;
};
-static MapMe MapMeDefault = {.retx = MAPME_DEFAULT_RETX,
- .Tu = MAPME_DEFAULT_TU,
- .removeFibEntries = false};
+#define NEXT_SLOT(CUR) (1 - CUR)
+#define CUR mapme->retx_array[mapme->cur]
+#define NXT mapme->retx_array[NEXT_SLOT(mapme->cur)]
+#define CURLEN mapme->retx_len[mapme->cur]
+#define NXTLEN mapme->retx_len[NEXT_SLOT(mapme->cur)]
+
+static mapme_t mapme_default = {
+ .retx = MAPME_DEFAULT_RETX,
+ .timescale = MAPME_DEFAULT_TU,
+ .discovery = MAPME_DEFAULT_DISCOVERY,
+ .protocol = MAPME_DEFAULT_PROTOCOL,
+ .enabled = true,
+ .timer = NULL,
+ // .retx_array = {{ 0 }}, // memset
+ .retx_len = {0},
+ .cur = 0, /* current slot */
+ .idle = 0,
+};
/******************************************************************************/
-bool mapme_create(MapMe **mapme, Forwarder *forwarder) {
- *mapme = malloc(sizeof(MapMe));
- if (!mapme) goto ERR_MALLOC;
-
- /* Internal state : set default values */
- memcpy(*mapme, &MapMeDefault, sizeof(MapMe));
+int mapme_on_timeout(void *mapme_arg, int fd, unsigned id, void *data);
- (*mapme)->forwarder = forwarder;
+mapme_t *mapme_create(void *forwarder) {
+ mapme_t *mapme = malloc(sizeof(mapme_t));
+ if (!mapme) return NULL;
- /* As there is no face table and no related events, we need to install hooks
- * in various places in the forwarder, where both control commands and
- * signalization are processed.
- */
+ /* Internal state : set default values */
+ memcpy(mapme, &mapme_default, sizeof(mapme_t));
+ memset(mapme->retx_array, 0, NUM_RETX_SLOT * NUM_RETX_ENTRIES);
- return true;
+ mapme->forwarder = forwarder;
+ loop_timer_create(&mapme->timer, MAIN_LOOP, mapme, mapme_on_timeout, NULL);
+ if (!mapme->timer) {
+ ERROR("Error allocating mapme timer.");
+ free(mapme);
+ return NULL;
+ }
-ERR_MALLOC:
- return false;
+ return mapme;
}
-void mapme_free(MapMe *mapme)
-{
- free(mapme);
+void mapme_free(mapme_t *mapme) {
+ loop_event_free(mapme->timer);
+ free(mapme);
}
/******************************************************************************
* TFIB
******************************************************************************/
-#define INVALID_SEQ 0
-#define INIT_SEQ 0
-
typedef struct {
+ // XXX We need magic number to know whether the TFIB was initialized or not
+ // ... or merge it inside the real data structure.
+ // NOTE: in VPP we reuse the nexthops in opposite order to gain room
+ // XXX need enough space in user_data !!
uint32_t seq;
- PARCHashMap *nexthops;
+ nexthops_t nexthops; // XXX useless shadow structure
/* Update/Notification heuristic */
- Ticks lastAckedUpdate;
-} MapMeTFIB;
+ Ticks last_acked_update;
+} mapme_tfib_t;
-static MapMeTFIB *mapmeTFIB_Create() {
- MapMeTFIB *tfib;
- tfib = malloc(sizeof(MapMeTFIB));
- if (!tfib) goto ERR_MALLOC;
+#define TFIB(FIB_ENTRY) ((mapme_tfib_t *)fib_entry_get_user_data(FIB_ENTRY))
+
+static mapme_tfib_t *mapme_tfib_create() {
+ mapme_tfib_t *tfib;
+ tfib = malloc(sizeof(mapme_tfib_t));
+ if (!tfib) return NULL;
+
+ // init
tfib->seq = INIT_SEQ;
- tfib->lastAckedUpdate = 0;
- tfib->nexthops = parcHashMap_Create();
- if (!tfib->nexthops) goto ERR_HASHMAP;
+ tfib->last_acked_update = 0;
+ nexthops_set_len(&tfib->nexthops, 0);
return tfib;
-
-ERR_HASHMAP:
- free(tfib);
-ERR_MALLOC:
- return NULL;
}
-void mapmeTFIB_Release(MapMeTFIB **tfibPtr) {
- MapMeTFIB *tfib = *tfibPtr;
- /* TODO; Release all timers */
- parcHashMap_Release(&tfib->nexthops);
+void mapme_release_tfib(mapme_tfib_t **tfibPtr) {
+ mapme_tfib_t *tfib = *tfibPtr;
free(tfib);
- *tfibPtr = NULL;
+ tfibPtr = NULL;
}
/**
- * @function mapme_CreateTFIB
- * @abstract Associate a new TFIB entry to a FIB entry.
+ * @function mapme_create_tfib
+ * @abstract Associate a new TFIB entry to a FIB entry. If a TFIB already exists
+ * the new one will be used
* @param [in] - Pointer to the FIB entry.
* @return Boolean indicating the success of the operation.
*/
-static void mapme_CreateTFIB(FibEntry *fibEntry) {
- MapMeTFIB *tfib;
-
- /* Make sure we don't already have an associated TFIB entry */
- tfib = fibEntry_getUserData(fibEntry);
- // assertNull(tfib);
+static void mapme_create_tfib(const mapme_t *mapme, fib_entry_t *entry) {
+ mapme_tfib_t *tfib;
- tfib = mapmeTFIB_Create();
- fibEntry_setUserData(fibEntry, tfib, (void (*)(void **))mapmeTFIB_Release);
+ tfib = mapme_tfib_create();
+ fib_entry_set_user_data(entry, tfib, (void (*)(void **))mapme_release_tfib);
}
-#define TFIB(fibEntry) ((MapMeTFIB *)fibEntry_getUserData(fibEntry))
-
-static const PARCEventTimer *mapmeTFIB_Get(const MapMeTFIB *tfib,
- unsigned conn_id) {
- const PARCEventTimer *timer;
- const PARCBuffer *buffer;
- PARCUnsigned *cid = parcUnsigned_Create(conn_id);
- buffer = parcHashMap_Get(tfib->nexthops, cid);
- if (!buffer) return NULL;
- PARCByteArray *array = parcBuffer_Array(buffer);
- timer = *((PARCEventTimer **)parcByteArray_Array(array));
- parcUnsigned_Release(&cid);
- return timer;
-}
-
-static void mapmeTFIB_Put(MapMeTFIB *tfib, unsigned conn_id,
- const PARCEventTimer *timer) {
- /* NOTE: Timers are not objects (the only class not being an object in
- * fact), and as such, we cannot use them as values for the HashMap.
- * Just like for unsigned we needed the PARC wrapper.
- * There is no wrapper for pointers, so we use Arrays, which has an ubly
- * syntax...
- */
- PARCUnsigned *cid = parcUnsigned_Create(conn_id);
- PARCBuffer *buffer =
- parcBuffer_CreateFromArray(&timer, sizeof(PARCEventTimer *));
- parcHashMap_Put(tfib->nexthops, cid, buffer);
- parcUnsigned_Release(&cid);
- parcBuffer_Release(&buffer);
-}
-
-static void mapmeTFIB_Remove(MapMeTFIB *tfib, unsigned conn_id) {
- // Who releases the timer ?
- PARCUnsigned *cid = parcUnsigned_Create(conn_id);
- parcHashMap_Remove(tfib->nexthops, cid);
- parcUnsigned_Release(&cid);
-}
-
-static PARCIterator *mapmeTFIB_CreateKeyIterator(const MapMeTFIB *tfib) {
- return parcHashMap_CreateKeyIterator(tfib->nexthops);
-}
-
-int hicn_prefix_from_name(const Name *name, hicn_prefix_t *prefix) {
- NameBitvector *bv = name_GetContentName(name);
- ip_prefix_t ip_prefix;
- nameBitvector_ToIPAddress(bv, &ip_prefix);
-
- /* The name length will be equal to ip address' prefix length */
- return hicn_prefix_create_from_ip_prefix(&ip_prefix, prefix);
+/**
+ * @brief Update/Notification heuristic:
+ *
+ * NOTE: IN are currently disabled until the proper placeholder is agreed in the
+ * interest header.
+ */
+static hicn_mapme_type_t mapme_get_type_from_heuristic(const mapme_t *mapme,
+ fib_entry_t *entry) {
+#if 0
+ if (!entry) return UPDATE;
+ if (fib_entry_has_local_nexthop(entry))
+ /* We are a producer for this entry, send update */
+ return UPDATE;
+#else /* Always send IU */
+ return UPDATE;
+#endif
}
-static Message *mapme_createMessage(const MapMe *mapme, const Name *name,
- mapme_params_t *params) {
- Ticks now = forwarder_GetTicks(mapme->forwarder);
- Logger *logger = logger_Acquire(forwarder_GetLogger(mapme->forwarder));
-
- INFO(mapme, "[MAP-Me] CreateMessage type=%d seq=%d", params->type,
- params->seq);
+/**
+ *
+ * Here nexthops is not necessarily FIB nexthops as we might advertise given FIB
+ * entries on various other connections.
+ *
+ * prefix can be specified to send an update for a More Specific Prefix (MSP),
+ * or left NULL for the default behaviour. Note there will be no support for
+ * retransmission for MSP.
+ *
+ * NOTES:
+ * - if the face is pending an we receive an IN, maybe we should not cancel the
+ * timer
+ * - this function should never be called for Notifications.
+ */
+int mapme_send_to_nexthops(const mapme_t *mapme, fib_entry_t *entry,
+ const nexthops_t *nexthops,
+ const hicn_prefix_t *prefix) {
+ INFO("mapme send to nexthops");
+ const hicn_prefix_t *mapme_prefix;
+ uint32_t mapme_seq;
+
+ assert(!!prefix ^ !!entry);
+
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return -1;
+ }
- size_t size = (params->protocol == IPPROTO_IPV6) ? HICN_MAPME_V6_HDRLEN
- : HICN_MAPME_V4_HDRLEN;
- uint8_t *icmp_pkt = parcMemory_AllocateAndClear(size);
+ if (prefix) {
+ INFO("mapme with given prefix");
+ mapme_prefix = prefix;
+ mapme_seq = 1;
+ } else {
+ INFO("mapme wih fib entry prefix");
+ mapme_tfib_t *tfib = TFIB(entry);
+ if (tfib == NULL) {
+ mapme_create_tfib(mapme, entry);
+ tfib = TFIB(entry);
+ }
- hicn_prefix_t prefix;
- int rc = hicn_prefix_from_name(name, &prefix);
- if (rc < 0) {
- ERR(mapme, "[MAP-Me] Failed to create lib's name");
- goto ERR_NAME;
+ mapme_prefix = fib_entry_get_prefix(entry);
+ mapme_seq = tfib->seq;
}
- INFO(mapme, "[MAP-Me] Creating MAP-Me packet");
- size_t len = hicn_mapme_create_packet(icmp_pkt, &prefix, params);
- if (len == 0) {
- ERR(mapme, "[MAP-Me] Failed to create mapme packet through lib");
- goto ERR_CREATE;
+ char prefix_s[MAXSZ_IP_PREFIX];
+ int rc = hicn_prefix_snprintf(prefix_s, MAXSZ_IP_PREFIX, mapme_prefix);
+ assert(rc < MAXSZ_IP_PREFIX);
+ if (rc < 0) NULL;
+
+ INFO("mapme send to nexthops prefix= %s", prefix_s);
+
+ WITH_INFO({
+ char buf[MAXSZ_HICN_PREFIX];
+ int rc = hicn_prefix_snprintf(buf, MAXSZ_HICN_PREFIX, mapme_prefix);
+ if (rc < 0 || rc >= MAXSZ_HICN_PREFIX)
+ snprintf(buf, MAXSZ_HICN_PREFIX, "(error)");
+ INFO("sending IU/IN for name %s on nexthops", buf);
+ })
+
+ mapme_params_t params = {.protocol = mapme->protocol,
+ .type = mapme_get_type_from_heuristic(mapme, entry),
+ .seq = mapme_seq};
+
+ uint8_t packet[MTU];
+ size_t size = hicn_mapme_create_packet(packet, mapme_prefix, &params);
+ if (size <= 0) {
+ ERROR("Could not create MAP-Me packet");
+ return -1;
}
- // hicn_packet_dump(icmp_pkt, MAPME_HDRLEN);
+ connection_table_t *table = forwarder_get_connection_table(mapme->forwarder);
- return message_CreateFromByteArray(NO_INGRESS, icmp_pkt,
- MessagePacketType_Interest, now, logger);
+ nexthops_foreach(nexthops, nexthop, {
+ const connection_t *conn = connection_table_get_by_id(table, nexthop);
+ assert(!connection_is_local(conn));
+ connection_send_packet(conn, packet, size);
+ });
-ERR_CREATE:
-ERR_NAME:
- return NULL;
+ return 0;
}
-static Message *mapme_createAckMessage(const MapMe *mapme,
- const uint8_t *msgBuffer,
- const mapme_params_t *params) {
- Ticks now = forwarder_GetTicks(mapme->forwarder);
- Logger *logger = logger_Acquire(forwarder_GetLogger(mapme->forwarder));
-
- size_t size = (params->protocol == IPPROTO_IPV6) ? HICN_MAPME_V6_HDRLEN
- : HICN_MAPME_V4_HDRLEN;
- uint8_t *icmp_pkt = parcMemory_AllocateAndClear(size);
- memcpy(icmp_pkt, msgBuffer, size);
-
- size_t len = hicn_mapme_create_ack(icmp_pkt, params);
- if (len != size) {
- ERR(mapme, "[MAP-Me] Failed to create mapme ack packet through lib");
- return NULL;
+#if 0
+/**
+ *
+ * Here nexthops is not necessarily FIB nexthops as we might advertise given FIB
+ * entries on various other connections.
+ */
+void mapme_maybe_send_to_nexthops(const mapme_t *mapme, fib_entry_t *fib_entry,
+ const nexthops_t *nexthops) {
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return;
}
- return message_CreateFromByteArray(
- NO_INGRESS, icmp_pkt, MessagePacketType_ContentObject, now, logger);
-}
-
-struct setFacePendingArgs {
- const MapMe *mapme;
- const Name *name;
- FibEntry *fibEntry;
- unsigned conn_id;
- bool send;
- bool is_producer;
- uint32_t num_retx;
-};
-
-static bool mapme_setFacePending(const MapMe *mapme, const Name *name,
- FibEntry *fibEntry, unsigned conn_id,
- bool send, bool is_producer, bool clear_tfib, uint32_t num_retx);
-
-static void mapme_setFacePendingCallback(int fd, PARCEventType which_event,
- void *data) {
- struct setFacePendingArgs *args = (struct setFacePendingArgs *)data;
-
- parcAssertTrue(which_event & PARCEventType_Timeout,
- "Event incorrect, expecting %X set, got %X",
- PARCEventType_Timeout, which_event);
+ /* Detect change */
+ if (!fib_entry_nexthops_changed(fib_entry)) {
+ INFO("No change in nexthops");
+ return;
+ }
+ fib_entry_set_prev_nexthops(fib_entry);
- INFO(args->mapme, "Timeout during retransmission. Re-sending");
- mapme_setFacePending(args->mapme, args->name, args->fibEntry, args->conn_id,
- args->send, args->is_producer, false, args->num_retx);
+ mapme_send_to_nexthops(mapme, fib_entry, nexthops);
}
-
-/**
- * @brief Update/Notification heuristic:
- *
- * NOTE: IN are currently disabled until the proper placeholder is agreed in the
- * interest header.
- */
-static hicn_mapme_type_t mapme_getTypeFromHeuristic(const MapMe *mapme,
- FibEntry *fibEntry) {
-#if 0 /* interplay of IU/IN */
- if (TFIB(fibEntry)->lastAckedUpdate == 0) {
- return UPDATE;
- } else {
- Ticks interval = now - TFIB(fibEntry)->lastAckedUpdate;
- return (T2NS(interval) > MS2NS(mapme->Tu)) ? UPDATE : NOTIFICATION;
- }
-#else /* Always send IU */
- return UPDATE;
#endif
-}
-static bool mapme_setFacePending(const MapMe *mapme, const Name *name,
- FibEntry *fibEntry, unsigned conn_id,
- bool send, bool is_producer, bool clear_tfib, uint32_t num_retx) {
- int rc;
+/******************************************************************************
+ * MAPME API
+ ******************************************************************************/
- INFO(mapme, "[MAP-Me] SetFacePending connection=%d prefix=XX retx=%d",
- conn_id, num_retx);
+int mapme_set_all_adjacencies(const mapme_t *mapme, fib_entry_t *entry) {
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return -1;
+ }
- /* NOTE: if the face is pending an we receive an IN, maybe we should not
- * cancel the timer
- */
- Dispatcher *dispatcher = forwarder_GetDispatcher(mapme->forwarder);
- PARCEventTimer *timer;
+ /* Apply the policy of the fib_entry over all neighbours */
+ nexthops_t new_nexthops = NEXTHOPS_EMPTY;
+ nexthops_t *nexthops = fib_entry_get_mapme_nexthops(entry, &new_nexthops);
- /* Safeguard during retransmissions */
- if (!TFIB(fibEntry))
- return true;
+ /* We set force to true to avoid overriding the FIB cache */
+ return mapme_set_adjacencies(mapme, entry, nexthops, NULL);
+}
+// XXX this will change with the FIB cache
+// XXX we are sometimes incrementing tfib seq for nothing
+int mapme_set_adjacencies(const mapme_t *mapme, fib_entry_t *entry,
+ nexthops_t *nexthops, const hicn_prefix_t *prefix) {
+ INFO("mapme set adjacenies");
/*
- * On the producer side, we have to clear the TFIB everytime we change the list
- * of adjacencies, otherwise retransmissions will occur to preserve them.
+ * - entry is provided in case of a producer reannouncement
+ * - prefix is provided for control plane triggered updates
*/
- if (clear_tfib) {
- /*
- * It is likely we cannot iterator and remove elements from the hashmap at
- * the same time, so we proceed in two steps
- */
- if (parcHashMap_Size(TFIB(fibEntry)->nexthops) > 0) {
+ assert(!!prefix ^ !!entry);
- NumberSet * conns = numberSet_Create();
-
- PARCIterator *it = parcHashMap_CreateKeyIterator(TFIB(fibEntry)->nexthops);
- while (parcIterator_HasNext(it)) {
- PARCUnsigned *cid = parcIterator_Next(it);
- unsigned conn_id = parcUnsigned_GetUnsigned(cid);
- numberSet_Add(conns, conn_id);
- }
- parcIterator_Release(&it);
-
- for (size_t i = 0; i < numberSet_Length(conns); i++) {
- unsigned conn_id = numberSet_GetItem(conns, i);
- PARCEventTimer *oldTimer = (PARCEventTimer *)mapmeTFIB_Get(TFIB(fibEntry), conn_id);
- if (oldTimer)
- parcEventTimer_Stop(oldTimer);
- mapmeTFIB_Remove(TFIB(fibEntry), conn_id);
- }
-
- numberSet_Release(&conns);
- }
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return -1;
}
- // NOTE
- // - at producer, send always true, we always send something reliably so we
- // set the timer.
- // - in the network, we always forward an IU, and never an IN
- //if (is_producer || send) {
- if (send) {
- mapme_params_t params = {
- .protocol = IPPROTO_IPV6,
- .type = is_producer ? mapme_getTypeFromHeuristic(mapme, fibEntry) : UPDATE,
- .seq = TFIB(fibEntry)->seq};
- Message *special_interest = mapme_createMessage(mapme, name, &params);
- if (!special_interest) {
- INFO(mapme, "[MAP-Me] Could not create special interest");
- return false;
- }
+ if (entry) {
+ /* Check disabled, we need to be able to send an update for a larger prefix
+ * that the one being served.
+ */
+ // if (!fib_entry_has_local_nexthop(entry)) return -1;
- const ConnectionTable *table =
- forwarder_GetConnectionTable(mapme->forwarder);
- const Connection *conn =
- connectionTable_FindById((ConnectionTable *)table, conn_id);
- if (conn) {
- const Name * name = message_GetName(special_interest);
- char * name_str = name_ToString(name);
- INFO(mapme, "[MAP-Me] Sending MAP-Me packet name=%s seq=%d conn=%d",
- name_str, params.seq, conn_id);
- free(name_str);
- connection_ReSend(conn, special_interest, NOT_A_NOTIFICATION);
- } else {
- INFO(mapme, "[MAP-Me] Stopped retransmissions as face went down");
+ mapme_tfib_t *tfib = TFIB(entry);
+ if (tfib == NULL) {
+ mapme_create_tfib(mapme, entry);
+ tfib = TFIB(entry);
}
- if (num_retx < MAX_RETX) {
- INFO(mapme, "[MAP-Me] - Scheduling retransmission\n");
- /* Schedule retransmission */
- struct setFacePendingArgs *args =
- malloc(sizeof(struct setFacePendingArgs));
- if (!args) goto ERR_MALLOC;
- args->mapme = mapme;
- args->name = name;
- args->fibEntry = fibEntry;
- args->conn_id = conn_id;
- args->send = send;
- args->is_producer = is_producer;
- args->num_retx = num_retx + 1;
-
- timer = dispatcher_CreateTimer(dispatcher, TIMER_NO_REPEAT,
- mapme_setFacePendingCallback, args);
- struct timeval timeout = {mapme->retx / 1000,
- (mapme->retx % 1000) * 1000};
- rc = parcEventTimer_Start(timer, &timeout);
- if (rc < 0) goto ERR_TIMER;
- } else {
- INFO(mapme, "[MAP-Me] Last retransmission.");
- timer = NULL;
- }
- } else {
- INFO(mapme, "[MAP-Me] - not forwarding as send is False");
- timer = NULL;
- }
+ /*
+ * We need to prevent pending updates to recreate a link which does not make
+ * since anymore since we edit the graph here.
+ */
+ nexthops_clear(&tfib->nexthops);
- PARCEventTimer *oldTimer =
- (PARCEventTimer *)mapmeTFIB_Get(TFIB(fibEntry), conn_id);
- if (oldTimer) {
- INFO(mapme, "[MAP-Me] - Found old timer, would need to cancel !");
- // parcEventTimer_Stop(oldTimer);
+ /* We update the sequence number in all cases otherwise this won't allow
+ * repetition
+ */
+ tfib->seq++;
}
- INFO(mapme, "[MAP-Me] - Putting new timer in TFIB");
- if (timer) mapmeTFIB_Put(TFIB(fibEntry), conn_id, timer);
- return true;
-
-ERR_MALLOC:
-ERR_TIMER:
- return false;
+ INFO("calling send to nh");
+ mapme_send_to_nexthops(mapme, entry, nexthops, prefix);
+ return 0;
}
-/*------------------------------------------------------------------------------
- * Event handling
- *----------------------------------------------------------------------------*/
+int mapme_set_adjacency(const mapme_t *mapme, fib_entry_t *entry,
+ nexthop_t nexthop, const hicn_prefix_t *prefix) {
+ nexthops_t nexthops = NEXTHOPS_EMPTY;
+ nexthops_add(&nexthops, nexthop);
-/*
- * Return true if we have at least one local connection as next hop
- */
-static bool mapme_hasLocalNextHops(const MapMe *mapme,
- const FibEntry *fibEntry) {
- const NumberSet *nexthops = fibEntry_GetNexthops(fibEntry);
- const ConnectionTable *table = forwarder_GetConnectionTable(mapme->forwarder);
-
- for (size_t j = 0; j < fibEntry_NexthopCount(fibEntry); j++) {
- /* Retrieve Nexthop #j */
- unsigned conn_id = numberSet_GetItem(nexthops, j);
- const Connection *conn =
- connectionTable_FindById((ConnectionTable *)table, conn_id);
-
- /* Ignore non-local connections */
- if (!connection_IsLocal(conn)) continue;
- /* We don't need to test against conn_added since we don't
- * expect it to have any entry in the FIB */
-
- return true;
- }
- return false;
+ return mapme_set_adjacencies(mapme, entry, &nexthops, prefix);
}
-void
-mapme_send_updates(const MapMe * mapme, FibEntry * fibEntry, const NumberSet * nexthops)
-{
- if (!TFIB(fibEntry)) /* Create TFIB associated to FIB entry */
- mapme_CreateTFIB(fibEntry);
- TFIB(fibEntry)->seq++;
-
- const Name *name = fibEntry_GetPrefix(fibEntry);
- char *name_str = name_ToString(name);
- bool clear_tfib = true;
- for (size_t j = 0; j < numberSet_Length(nexthops); j++) {
- unsigned nexthop_id = numberSet_GetItem(nexthops, j);
- INFO(mapme, "[MAP-Me] sending IU/IN for name %s on connection %d", name_str,
- nexthop_id);
- mapme_setFacePending(mapme, name, fibEntry, nexthop_id, true, true, clear_tfib, 0);
- clear_tfib = false;
+int mapme_update_adjacencies(const mapme_t *mapme, fib_entry_t *entry,
+ bool inc_iu_seq) {
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return -1;
}
- free(name_str);
-}
-
-void
-mapme_maybe_send_updates(const MapMe * mapme, FibEntry * fibEntry, const NumberSet * nexthops)
-{
- /* Detect change */
- NumberSet * previous_nexthops = fibEntry_GetPreviousNextHops(fibEntry);
- if (numberSet_Equals(nexthops, previous_nexthops)) {
- INFO(mapme, "[MAP-Me] No change in nexthops");
- return;
+ mapme_tfib_t *tfib = TFIB(entry);
+ if (tfib == NULL) {
+ mapme_create_tfib(mapme, entry);
+ tfib = TFIB(entry);
}
- fibEntry_SetPreviousNextHops(fibEntry, nexthops);
- mapme_send_updates(mapme, fibEntry, nexthops);
-}
+ if (inc_iu_seq) tfib->seq++;
-void
-mapme_reconsiderFibEntry(const MapMe *mapme, FibEntry * fibEntry)
-{
- /*
- * Skip entries that do not correspond to a producer ( / have a locally
- * served prefix / have no local connection as next hop)
- */
- if (!mapme_hasLocalNextHops(mapme, fibEntry))
- return;
+ mapme_send_to_nexthops(mapme, entry, &tfib->nexthops, NULL);
+ return 0;
+}
- /* Apply the policy of the fibEntry over all neighbours */
- NumberSet * available_nexthops = fibEntry_GetAvailableNextHops(fibEntry, ~0);
+int mapme_send_to_nexthop(const mapme_t *mapme, fib_entry_t *entry,
+ unsigned nexthop) {
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return -1;
+ }
- /* Advertise prefix on all available next hops (if needed) */
- mapme_send_updates(mapme, fibEntry, available_nexthops);
+ nexthops_t nexthops = NEXTHOPS_EMPTY;
+ nexthops_add(&nexthops, nexthop);
- numberSet_Release(&available_nexthops);
+ return mapme_send_to_nexthops(mapme, entry, &nexthops, NULL);
}
+#if 0
/*
* Callback called everytime a new connection is created by the control protocol
*/
-void
-mapme_onConnectionEvent(const MapMe *mapme, const Connection *conn_added, connection_event_t event) {
+void mapme_on_connection_event(const mapme_t *mapme,
+ const connection_t *conn_added,
+ connection_event_t event) {
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return;
+ }
+
/* Does the priority change impacts the default route selection; if so,
* advertise the prefix on this default route. If there are many default
* routes, either v4 v6, or many connections as next hops on this default
* route, then send to all.
*/
if (conn_added) {
- if (connection_IsLocal(conn_added))
- return;
+ if (connection_is_local(conn_added)) return;
- unsigned conn_added_id = connection_GetConnectionId(conn_added);
- switch(event) {
+ unsigned conn_added_id = connection_get_id(conn_added);
+ switch (event) {
case CONNECTION_EVENT_CREATE:
- INFO(mapme, "[MAP-Me] Connection %d got created", conn_added_id);
- break;
+ INFO("Connection %d got created", conn_added_id);
+ break;
case CONNECTION_EVENT_DELETE:
- INFO(mapme, "[MAP-Me] Connection %d got deleted", conn_added_id);
- break;
+ INFO("Connection %d got deleted", conn_added_id);
+ break;
case CONNECTION_EVENT_UPDATE:
- INFO(mapme, "[MAP-Me] Connection %d got updated", conn_added_id);
- break;
+ INFO("Connection %d got updated", conn_added_id);
+ break;
case CONNECTION_EVENT_SET_UP:
- INFO(mapme, "[MAP-Me] Connection %d went up", conn_added_id);
- break;
+ INFO("Connection %d went up", conn_added_id);
+ break;
case CONNECTION_EVENT_SET_DOWN:
- INFO(mapme, "[MAP-Me] Connection %d went down", conn_added_id);
- break;
+ INFO("Connection %d went down", conn_added_id);
+ break;
case CONNECTION_EVENT_TAGS_CHANGED:
- INFO(mapme, "[MAP-Me] Connection %d changed tags", conn_added_id);
- break;
+ INFO("Connection %d changed tags", conn_added_id);
+ break;
case CONNECTION_EVENT_PRIORITY_CHANGED:
- INFO(mapme, "[MAP-Me] Connection %d changed priority to %d",
- conn_added_id, connection_GetPriority(conn_added));
- break;
+ INFO("Connection %d changed priority to %d", conn_added_id,
+ connection_get_priority(conn_added));
+ break;
}
}
/* We need to send a MapMe update on the newly selected connections for
- * each concerned fibEntry : connection is involved, or no more involved */
- FibEntryList *fiblist = forwarder_GetFibEntries(mapme->forwarder);
+ * each concerned fib_entry : connection is involved, or no more involved */
+ const fib_t *fib = forwarder_get_fib(mapme->forwarder);
+ fib_foreach_entry(fib, entry, { mapme_set_all_adjacencies(mapme, entry); });
+}
+#endif
- /* Iterate a first time on the FIB to get the locally served prefixes */
- for (size_t i = 0; i < fibEntryList_Length(fiblist); i++) {
- FibEntry *fibEntry = (FibEntry *)fibEntryList_Get(fiblist, i);
- mapme_reconsiderFibEntry(mapme, fibEntry);
+/*------------------------------------------------------------------------------
+ * Special Interest handling
+ *----------------------------------------------------------------------------*/
+
+#ifdef HICN_MAPME_ALLOW_NONEXISTING_FIB_ENTRY
+fib_entry_t *mapme_create_fib_entry(const mapme_t *mapme,
+ const hicn_prefix_t *prefix,
+ unsigned ingress_id) {
+ INFO(" - creating FIB entry with next hop on connection %d", ingress_id);
+
+ /*
+ * This might happen for a node hosting a producer which has moved.
+ * Destroying the face has led to removing all corresponding FIB
+ * entries. In that case, we need to correctly restore the FIB entries.
+ * Also in case we have an intermediate node with just a less specific prefix
+ * (eg. a default route), and thus an announcement with a more specific
+ * prefix. In that case we need to perform a FIB lookup to find the next hops
+ * to which the message should be propagated (before adding).
+ */
+ forwarder_t *forwarder = mapme->forwarder;
+
+ fib_t *fib = forwarder_get_fib(forwarder);
+ configuration_t *config = forwarder_get_configuration(forwarder);
+
+ char prefix_s[MAXSZ_IP_PREFIX];
+ int rc = hicn_prefix_snprintf(prefix_s, MAXSZ_IP_PREFIX, prefix);
+ assert(rc < MAXSZ_IP_PREFIX);
+ if (rc < 0) NULL;
+
+ INFO("creating FIB entry for prefix %s", prefix_s);
+
+ strategy_type_t strategy_type = configuration_get_strategy(config, prefix_s);
+ fib_entry_t *entry = fib_entry_create(prefix, strategy_type, NULL, forwarder);
+ mapme_create_tfib(mapme, entry);
+
+ fib_entry_t *lpm = fib_match_prefix(fib, prefix);
+
+ // Keep this after the LPM lookup
+ fib_add(fib, entry);
+
+ if (!lpm) {
+ TFIB(entry)->seq = 0;
+ if (ingress_id != INVALID_FACE_ID)
+ fib_entry_nexthops_add(entry, ingress_id);
+ return entry;
}
- fibEntryList_Destroy(&fiblist);
+ /*
+ * We make a clone of the FIB entry (zero'ing the sequence number ?) with
+ * the more specific name, and proceed as usual. Worst case we clone the
+ * default route...
+ */
+ const nexthops_t *lpm_nexthops = fib_entry_get_nexthops(lpm);
+ nexthops_foreach(lpm_nexthops, nh, { fib_entry_nexthops_add(entry, nh); });
+ return entry;
+}
+#endif
+
+int mapme_on_timeout(void *mapme_arg, int fd, unsigned id, void *data) {
+ mapme_t *mapme = mapme_arg;
+ assert(mapme);
+ assert(id == 0);
+ assert(!data);
+ /* Timeout occurred, we have to retransmit IUs for all pending
+ * prefixes having entries in TFIB
+ *
+ * timeouts are slotted
+ * | | | |
+ *
+ * ^
+ * +- event occurred
+ * new face, wait for the second next
+ * (having two arrays and swapping cur and next)
+ * retx : put in next
+ */
+ mapme->idle += 1;
- INFO(mapme, "[MAP-Me] Done");
+ for (uint8_t pos = 0; pos < CURLEN; pos++) {
+ mapme_retx_t *retx = &CUR[pos];
+
+ if (!retx->entry) /* deleted entry */
+ continue;
+
+ mapme_tfib_t *tfib = TFIB(retx->entry);
+ assert(tfib);
+
+ /* Re-send interest for all entries */
+ mapme_update_adjacencies(mapme, retx->entry, false);
+
+ retx->retx_count++;
+ /* If we exceed the numver of retransmittion it means that all tfib
+ * entries have seens at least HICN_PARAM_RETX_MAX of retransmission
+ */
+ if (retx->retx_count < MAPME_MAX_RETX) {
+ /*
+ * We did some retransmissions, so let's reschedule a check in the
+ * next slot
+ */
+ NXT[NXTLEN++] = CUR[pos];
+ mapme->idle = 0;
+ } else {
+ WARN("Maximum retransmissions exceeded");
+ /* If we exceed the numver of retransmission it means that all TFIB
+ * entries have seens at least HICN_PARAM_RTX_MAX retransmissions.
+ * (Deletion might be slightly late).
+ *
+ * XXX document: when adding an entry in TFIB, we might exceed max
+ * retransmissions for previous entries that started retransmitting
+ * beforehand.
+ */
+ nexthops_clear(&tfib->nexthops);
+ }
+ }
+
+ /* Reset events in this slot and prepare for next one */
+ CURLEN = 0;
+ mapme->cur = NEXT_SLOT(mapme->cur);
+
+ /* After two empty slots, we disable the timer */
+ if (mapme->idle > 1) {
+ loop_event_unregister(mapme->timer);
+ }
+
+ return 0;
}
+static void mapme_on_event(mapme_t *mapme, mapme_event_t event,
+ fib_entry_t *entry, unsigned ingress_id) {
+ switch (event) {
#if 0
-#ifdef WITH_POLICY
-void mapme_onPolicyUpdate(const MapMe *mapme, const Connection *conn_selected, FibEntry * fibEntry)
-{
- /* Ignore local connections corresponding to applications for now */
- if (connection_IsLocal(conn_selected))
- return;
+ case HICN_MAPME_EVENT_FACE_ADD:
+ {
+ /*
+ * A face has been added:
+ * - In case of a local app face, we need to advertise a new prefix
+ * - For another local face type, we need to advertise local
+ * prefixes and schedule retransmissions
+ */
+ mapme_retx_t *retx_events = event_data;
+ for (uint8_t i = 0; i < vec_len (retx_events); i++) {
+ hicn_mapme_on_face_added(mapme, retx_events[i].dpo);
+ }
+
+ if (mapme->timer_fd == -1)
+ mapme->timer_fd = loop_register_timer(MAIN_LOOP,
+ mapme->retx, mapme, mapme_on_timeout, NULL);
+ mapme->idle = 0;
+ break;
+ }
+ case HICN_MAPME_EVENT_FACE_DEL:
+ if (mapme->timer_fd == -1)
+ mapme->timer_fd = loop_register_timer(MAIN_LOOP,
+ DEFAULT_TIMEOUT, mapme, mapme_on_timeout, NULL);
+ mapme->idle = 0;
+ break;
+#endif
- unsigned conn_selected_id = connection_GetConnectionId(conn_selected);
- INFO(mapme, "[MAP-Me] New connection %d", conn_selected_id);
+ case MAPME_EVENT_NH_SET:
+ /*
+ * An hICN FIB entry has been modified. All operations so far
+ * have been procedded in the nodes. Here we need to track
+ * retransmissions upon timeout: we mark the FIB entry as pending in
+ * the second-to-next slot
+ */
- const Name *name = fibEntry_GetPrefix(fibEntry);
+ /*
+ * XXX Move this in doc
+ *
+ * The FIB entry has a new next hop, and its TFIB section has:
+ * - eventually previous prev hops for which a IU with a
+ * lower seqno has been sent
+ * - the prev hops that have just been added.
+ *
+ * We don't distinguish any and just send an updated IU to all
+ * of them. The retransmission of the latest IU to all
+ * facilitates the matching of ACKs to a single seqno which is
+ * the one stored in the FIB.
+ *
+ * Since we retransmit to all prev hops, we can remove this
+ * (T)FIB entry for the check at the end of the current slot.
+ */
- /* Skip entries that have no local connection as next hop */
- if (!mapme_hasLocalNextHops(mapme, fibEntry))
- return;
+ /* Mark FIB entry as pending for second-to-next slot */
+ /*
+ * Transmit IU for all TFIB entries with latest seqno (we have
+ * at least one for sure!)
+ */
+ mapme_update_adjacencies(mapme, entry, false);
+
+ /* Delete entry_id from retransmissions in the current slot (if present)
+ * ... */
+ /* ... and schedule it for next slot (if not already) */
+ uint8_t j;
+ for (j = 0; j < CURLEN; j++) {
+ if (CUR[j].entry == entry) CUR[j].entry = NULL; /* sufficient */
+ }
+ for (j = 0; j < NXTLEN; j++) {
+ if (NXT[j].entry == entry) break;
+ }
+ if (j == NXTLEN) /* not found */
+ NXT[NXTLEN++] = (mapme_retx_t){
+ .entry = entry,
+ .retx_count = 0,
+ };
+
+ if (!loop_timer_is_enabled(mapme->timer)) {
+ if (loop_timer_register(mapme->timer, mapme->retx) < 0) {
+ ERROR("Error setting mapme timer.");
+ break;
+ }
+ }
+ mapme->idle = 0;
+ break;
- /* This entry corresponds to a locally served prefix, set
- * Special Interest */
- if (!TFIB(fibEntry)) /* Create TFIB associated to FIB entry */
- mapme_CreateTFIB(fibEntry);
- TFIB(fibEntry)->seq++;
+ case MAPME_EVENT_NH_ADD:
+ /*
+ * XXX move this in doc
+ *
+ * As per the description of states, this event should add the face
+ * to the list of next hops, and eventually remove it from TFIB.
+ * This corresponds to the multipath case.
+ *
+ * In all cases, we assume the propagation was already done when the first
+ * interest with the same sequence number was received, so we stop here
+ * No change in TFIB = no IU to send
+ *
+ * No change in timers.
+ */
- char *name_str = name_ToString(name);
- INFO(mapme, "[MAP-Me] sending IU/IN for name %s on connection %d", name_str,
- conn_selected_id);
- free(name_str);
+ // XXX useless
+#if 0
+ /* Add ingress face as next hop */
+ idle = 0;
+#endif
+ break;
- mapme_setFacePending(mapme, name, fibEntry, conn_selected_id, true, true, true, 0);
+ case MAPME_EVENT_PH_ADD:
+ /* Back-propagation, interesting even for IN (desync) */
+ mapme_send_to_nexthop(mapme, entry, ingress_id);
+
+ mapme->idle = 0;
+ if (!loop_timer_is_enabled(mapme->timer))
+ loop_timer_register(mapme->timer, mapme->retx);
+ break;
+
+ case MAPME_EVENT_PH_DEL:
+ /* Ack : remove an element from TFIB */
+ break;
+
+ case MAPME_EVENT_FACE_ADD:
+ case MAPME_EVENT_FACE_DEL:
+
+ case MAPME_EVENT_UNDEFINED:
+ case MAPME_EVENT_N:
+ ERROR("Unexpected event");
+ break;
+ }
}
-#endif /* WITH_POLICY */
-#endif
-/*------------------------------------------------------------------------------
- * Special Interest handling
- *----------------------------------------------------------------------------*/
+static void mapme_on_interest(mapme_t *mapme, msgbuf_t *msgbuf,
+ unsigned ingress_id, hicn_prefix_t *prefix,
+ mapme_params_t *params) {
+ connection_table_t *table = forwarder_get_connection_table(mapme->forwarder);
-/**
- * @discussion This function is way too long and should be cut out
- */
-static bool mapme_onSpecialInterest(const MapMe *mapme,
- const uint8_t *msgBuffer,
- unsigned conn_in_id, hicn_prefix_t *prefix,
- mapme_params_t *params) {
- const ConnectionTable *table = forwarder_GetConnectionTable(mapme->forwarder);
/* The cast is needed since connectionTable_FindById miss the
* const qualifier for the first parameter */
- const Connection *conn_in =
- connectionTable_FindById((ConnectionTable *)table, conn_in_id);
- seq_t fibSeq, seq = params->seq;
- bool send = (params->type == UPDATE);
- bool rv;
-
- Name *name = name_CreateFromPacket(msgBuffer, MessagePacketType_Interest);
- name_setLen(name, prefix->len);
- char *name_str = name_ToString(name);
- INFO(mapme,
- "[MAP-Me] Ack'ed Special Interest on connection %d - prefix=%s type=XX "
- "seq=%d",
- conn_in_id, name_str, seq);
- free(name_str);
+ const connection_t *conn_in = connection_table_get_by_id(table, ingress_id);
/*
* Immediately send an acknowledgement back on the ingress connection
- * We always ack, even duplicates.
+ * We always ack, even duplicates. Clone mgsbuf to avoid to overwrite the
+ * received message
*/
- Message *ack = mapme_createAckMessage(mapme, msgBuffer, params);
- if (!ack) goto ERR_ACK_CREATE;
- rv = connection_ReSend(conn_in, ack, NOT_A_NOTIFICATION);
- if (!rv) goto ERR_ACK_SEND;
- message_Release(&ack);
-
- /* EPM on FIB */
- /* only the processor has access to the FIB */
- FIB *fib = forwarder_getFib(mapme->forwarder);
+ msgbuf_t *ack;
+ msgbuf_pool_t *msgbuf_pool = forwarder_get_msgbuf_pool(mapme->forwarder);
+ off_t interest_offset = msgbuf_pool_get_id(msgbuf_pool, (msgbuf_t *)msgbuf);
+ msgbuf_pool_clone(msgbuf_pool, &ack, interest_offset);
+
+ uint8_t *ack_packet = msgbuf_get_packet(ack);
+ size_t size = hicn_mapme_create_ack(ack_packet, params);
+ if (!connection_send_packet(conn_in, ack_packet, size)) {
+ /* We accept the packet knowing we will get a retransmit */
+ ERROR("Failed to send ACK packet");
+ }
- FibEntry *fibEntry = fib_Contains(fib, name);
- if (!fibEntry) {
- INFO(mapme, "Ignored update with no FIB entry");
- return 0;
-#if 0
- INFO(mapme,
- "[MAP-Me] - Re-creating FIB entry with next hop on connection %d",
- conn_in_id);
- /*
- * This might happen for a node hosting a producer which has moved.
- * Destroying the face has led to removing all corresponding FIB
- * entries. In that case, we need to correctly restore the FIB entries.
- */
- strategy_type fwdStrategy = LAST_STRATEGY_VALUE;
+ msgbuf_pool_put(msgbuf_pool, ack);
- /*
- * It might also be due to the announcement of a more specific prefix. In
- * that case we need to perform a FIB lookup to find the next hops to which
- * the message should be propagated.
- */
-#ifdef WITH_POLICY
- fibEntry = fibEntry_Create(name, fwdStrategy, mapme->forwarder);
-#else
- fibEntry = fibEntry_Create(name, fwdStrategy);
-#endif /* WITH_POLICY */
- FibEntry *lpm = fib_MatchName(fib, name);
- mapme_CreateTFIB(fibEntry);
- fib_Add(fib, fibEntry);
- if (!lpm) {
- TFIB(fibEntry)->seq = seq;
- fibEntry_AddNexthop(fibEntry, conn_in_id);
- return true;
- }
+ WITH_INFO({
+ char buf[MAXSZ_HICN_PREFIX];
+ int rc = hicn_prefix_snprintf(buf, MAXSZ_HICN_PREFIX, prefix);
+ if (rc < 0 || rc >= MAXSZ_HICN_PREFIX)
+ snprintf(buf, MAXSZ_HICN_PREFIX, "%s", "(error)");
+ INFO("Ack'ed interest : connection=%d prefix=%s seq=%d", ingress_id, buf,
+ params->seq);
+ });
- /*
- * We make a clone of the FIB entry (zero'ing the sequence number ?) with
- * the more specific name, and proceed as usual. Worst case we clone the
- * default route...
- */
- const NumberSet *lpm_nexthops = fibEntry_GetNexthops(lpm);
- for (size_t i = 0; i < numberSet_Length(lpm_nexthops); i++) {
- fibEntry_AddNexthop(fibEntry, numberSet_GetItem(lpm_nexthops, i));
+ /* EPM on FIB */
+ const fib_t *fib = forwarder_get_fib(mapme->forwarder);
+ fib_entry_t *entry = fib_contains(fib, prefix);
+ if (!entry) {
+#ifdef HICN_MAPME_ALLOW_NONEXISTING_FIB_ENTRY
+ entry = mapme_create_fib_entry(mapme, prefix, ingress_id);
+ if (!entry) {
+ ERROR("Failed to create FIB entry");
+ return;
}
+#else
+ INFO("Ignored update with no FIB entry");
+ return;
#endif
+ }
- } else if (!TFIB(fibEntry)) {
- /* Create TFIB associated to FIB entry */
- INFO(mapme,
- "[MAP-Me] - Creating TFIB entry with default sequence number");
- mapme_CreateTFIB(fibEntry);
+ mapme_tfib_t *tfib = TFIB(entry);
+ if (tfib == NULL) {
+ mapme_create_tfib(mapme, entry);
+ tfib = TFIB(entry);
}
/*
@@ -725,108 +899,57 @@ static bool mapme_onSpecialInterest(const MapMe *mapme,
* Detection: we receive a message initially sent by ourselves, ie a message
* for which the prefix has a local next hop in the FIB.
*/
- if (mapme_hasLocalNextHops(mapme, fibEntry)) {
- INFO(mapme, "[MAP-Me] - Received original interest... Update complete");
- return true;
- }
+ // XXX NOT IN VPP ?
- fibSeq = TFIB(fibEntry)->seq;
- if (seq > fibSeq) {
- INFO(mapme,
- "[MAP-Me] - Higher sequence number than FIB %d, updating seq and "
- "next hops",
- fibSeq);
- /* This has to be done first to allow processing SpecialInterestAck's */
- TFIB(fibEntry)->seq = seq;
-
- /* Reliably forward the IU on all prevHops */
- INFO(mapme, "[MAP-Me] - (1/3) processing prev hops");
- if (params->type == UPDATE) {
- PARCIterator *iterator = mapmeTFIB_CreateKeyIterator(TFIB(fibEntry));
- while (parcIterator_HasNext(iterator)) {
- PARCUnsigned *cid = parcIterator_Next(iterator);
- unsigned conn_id = parcUnsigned_GetUnsigned(cid);
- INFO(mapme, "[MAP-Me] - Re-sending IU to pending connection %d",
- conn_id);
- mapme_setFacePending(mapme, fibEntry_GetPrefix(fibEntry), fibEntry,
- conn_id, false, false, false, 0);
- }
- parcIterator_Release(&iterator);
- }
+ /* Initially we were detecting that the update completed when it returned to
+ * the producer, Because there might be proxies (using local/remote strategy),
+ * we need instead to verify if there is no non-local faces.
+ */
- /* nextHops -> prevHops
- *
- * We add to the list of pendingUpdates the current next hops, and
- * eventually forward them an IU too.
- *
- * Exception: nextHops -> nextHops
- * Because of retransmission issues, it is possible that a second interest
- * (with same of higher sequence number) is receive from a next-hop
- * interface. In that case, the face remains a next hop.
- */
- const NumberSet *nexthops_old = fibEntry_GetNexthops(fibEntry);
+ if (fib_entry_has_all_local_nexthops(entry)) {
+ INFO("Received original interest... Update complete");
+ return;
+ }
- /* We make a copy to be able to send IU _after_ updating next hops */
- NumberSet *nexthops = numberSet_Create();
- numberSet_AddSet(nexthops, nexthops_old);
+ mapme_event_t event = MAPME_EVENT_UNDEFINED;
+ if (params->seq > tfib->seq) {
+ INFO(
+ "MAPME IU seq %d > fib_seq %d, updating seq and next hops, new "
+ "nexthop=%d",
+ params->seq, tfib->seq, ingress_id);
+ /* This has to be done first to allow processing ack */
+ // XXX this should even be done before sending ack, as in VPP.
+ tfib->seq = params->seq;
- /* We are considering : * -> nextHops
- *
- * If inFace was a previous hop, we need to cancel the timer and remove
- * the entry. Also, the face should be added to next hops.
+ /*
+ * Move nexthops to TFIB... but ingress_id that lands in nexthops
*
- * Optimization : nextHops -> nextHops
- * - no next hop to add
- * - we know that inFace was not a previous hop since it was a next hop and
- * this forms a partition. No need for a search
+ * This could might optimized for situations where nothing changes, but
+ * this is very unlikely if not impossible...
*/
-
- INFO(mapme, "[MAP-Me] - (3/3) next hops ~~> prev hops");
- PARCEventTimer *oldTimer =
- (PARCEventTimer *)mapmeTFIB_Get(TFIB(fibEntry), conn_in_id);
- if (oldTimer) {
- /* This happens if we receive an IU while we are still sending
- * one in the other direction
- */
- INFO(mapme, "[MAP-Me] - Canceled pending timer");
- parcEventTimer_Stop(oldTimer);
- }
- mapmeTFIB_Remove(TFIB(fibEntry), conn_in_id);
-
- /* Remove all next hops */
- for (size_t k = 0; k < numberSet_Length(nexthops); k++) {
- unsigned conn_id = numberSet_GetItem(nexthops, k);
- INFO(mapme, "[MAP-Me] - Replaced next hops by connection %d", conn_id);
- fibEntry_RemoveNexthopByConnectionId(fibEntry, conn_id);
- }
- fibEntry_AddNexthop(fibEntry, conn_in_id);
-
- INFO(mapme, "[MAP-Me] - (2/3) processing next hops");
- bool complete = true;
- for (size_t k = 0; k < numberSet_Length(nexthops); k++) {
- unsigned conn_id = numberSet_GetItem(nexthops, k);
- INFO(mapme, " - Next hop connection %d", conn_id);
- if (conn_id == conn_in_id) {
- INFO(mapme, " . Ignored this next hop since equal to ingress face");
- continue;
+ nexthops_t nexthops_keep = NEXTHOPS_EMPTY;
+ nexthops_foreach(&entry->nexthops, prevhop, {
+ const connection_t *conn = connection_table_get_by_id(table, prevhop);
+ /* Preserve local connections, migrate others to TFIB */
+ if (connection_is_local(conn)) {
+ nexthops_add(&nexthops_keep, prevhop);
+ } else {
+ nexthops_add(&tfib->nexthops, prevhop);
}
+ });
- INFO(mapme, "[MAP-Me] - Sending IU on current next hop connection %d",
- conn_id);
- mapme_setFacePending(mapme, fibEntry_GetPrefix(fibEntry), fibEntry,
- conn_id, send, false, false, 0);
- complete = false;
- }
+ nexthops_remove(&tfib->nexthops, ingress_id);
- /*
- * The update is completed when the IU could not be sent to any
- * other next hop.
- */
- if (complete) INFO(mapme, "[MAP-Me] - Update completed !");
+ nexthops_clear(&entry->nexthops);
+ nexthops_add(&entry->nexthops, ingress_id);
+ nexthops_foreach(&nexthops_keep, nh,
+ { nexthops_add(&entry->nexthops, nh); });
- numberSet_Release(&nexthops);
+ event = MAPME_EVENT_NH_SET;
- } else if (seq == fibSeq) {
+ // XXX tell things are complete if we have no IU to send
+
+ } else if (params->seq == tfib->seq) {
/*
* Multipath, multihoming, multiple producers or duplicate interest
*
@@ -837,145 +960,77 @@ static bool mapme_onSpecialInterest(const MapMe *mapme,
* producer and that we received back our own IU. In that case, we just
* need to Ack and ignore it.
*/
-#if 0
- if (mapme_hasLocalNextHops(mapme, fibEntry)) {
- INFO(mapme, "[MAP-Me] - Received original interest... Update complete");
- return true;
- }
-#endif
+ DEBUG("params.seq %d == fib_seq %d, adding nethop %d", params->seq,
+ tfib->seq, ingress_id);
+
+ /* Move ingress to nexthops (and eventually remove it from TFIB) */
+ nexthops_add(&entry->nexthops, ingress_id);
+ nexthops_remove(&tfib->nexthops, ingress_id);
- INFO(mapme, "[MAP-Me] - Adding multipath next hop on connection %d",
- conn_in_id);
- fibEntry_AddNexthop(fibEntry, conn_in_id);
+ event = MAPME_EVENT_NH_ADD;
- } else { // seq < fibSeq
+ } else { // params->seq < tfib->seq
/*
* Face is propagating outdated information, we can just
* consider it as a prevHops. Send the special interest backwards with
* the new sequence number to reconciliate this outdated part of the
* arborescence.
*/
- INFO(
- mapme,
- "[MAP-Me] - Update interest %d -> %d sent backwards on connection %d",
- seq, fibSeq, conn_in_id);
- mapme_setFacePending(mapme, fibEntry_GetPrefix(fibEntry), fibEntry,
- conn_in_id, send, false, false, 0);
+ if (nexthops_contains(&entry->nexthops, ingress_id)) {
+ INFO("Ignored seq %d < fib_seq %d from current nexthop on face %d",
+ params->seq, tfib->seq, ingress_id);
+ return;
+ } else {
+ INFO("Received seq %d < fib_seq %d, sending backwards on face %d",
+ params->seq, tfib->seq, ingress_id);
+ nexthops_add(&tfib->nexthops, ingress_id);
+ }
+
+ event = MAPME_EVENT_PH_ADD;
}
- return true;
+ /* Don't trigger events for notification unless we need to send interests
+ * backwards */
+ if ((params->type != UPDATE) && (event != MAPME_EVENT_PH_ADD)) return;
-ERR_ACK_SEND:
- message_Release(&ack);
-ERR_ACK_CREATE:
- return false;
+ mapme_on_event(mapme, event, entry, ingress_id);
}
-void mapme_onSpecialInterestAck(const MapMe *mapme, const uint8_t *msgBuffer,
- unsigned conn_in_id, hicn_prefix_t *prefix,
- mapme_params_t *params) {
- INFO(mapme, "[MAP-Me] Receive IU/IN Ack on connection %d", conn_in_id);
-
- const Name * name =
- name_CreateFromPacket(msgBuffer, MessagePacketType_ContentObject);
- name_setLen((Name*) name, prefix->len);
- char * name_str = name_ToString(name);
- INFO(mapme, "[MAP-Me] Received ack for name prefix=%s seq=%d on conn id=%d",
- name_str, params->seq, conn_in_id);
- free(name_str);
-
- FIB *fib = forwarder_getFib(mapme->forwarder);
- FibEntry *fibEntry = fib_Contains(fib, name);
- if (!fibEntry) {
+static void mapme_on_data(mapme_t *mapme, msgbuf_t *msgbuf, unsigned ingress_id,
+ hicn_prefix_t *prefix, mapme_params_t *params) {
+ WITH_INFO({
+ char buf[MAXSZ_HICN_PREFIX];
+ int rc = hicn_prefix_snprintf(buf, MAXSZ_HICN_PREFIX, prefix);
+ if (rc < 0 || rc >= MAXSZ_HICN_PREFIX)
+ snprintf(buf, MAXSZ_HICN_PREFIX, "(error)");
+ INFO("Received ack for name prefix=%s seq=%d on conn id=%d", buf,
+ params->seq, ingress_id);
+ })
+
+ const fib_t *fib = forwarder_get_fib(mapme->forwarder);
+ fib_entry_t *entry = fib_contains(fib, prefix);
+ if (!entry) {
+ INFO("Ignored ACK with no corresponding FIB entry");
return;
}
- parcAssertNotNull(fibEntry,
- "No corresponding FIB entry for name contained in IU Ack");
-
- /* Test if the latest pending update has been ack'ed, otherwise just ignore */
- seq_t seq = params->seq;
- if (seq != INVALID_SEQ) {
- seq_t fibSeq = TFIB(fibEntry)->seq;
-
- if (seq < fibSeq) {
-
- /* If we receive an old ack:
- * - either the connection is still a next hop and we have to ignore
- * the ack until we receive a further update with higher seqno
- * - or the connection is no more to be informed and the ack is
- * sufficient and we can remove future retransmissions
- */
-
- INFO(mapme,
- "[MAP-Me] - Ignored special interest Ack with seq=%u, expected %u",
- seq, fibSeq);
- return;
- }
- }
+ mapme_tfib_t *tfib = TFIB(entry);
/*
- * Ignore the Ack if no TFIB is present, or it has no corresponding entry
- * with the ingress face.
- * Note: previously, we were creating the TFIB entry
+ * As we always retransmit IU with the latest seq, we are not interested in
+ * ACKs with inferior seq
*/
- if (!TFIB(fibEntry)) {
- INFO(mapme, "[MAP-Me] - Ignored ACK for prefix with no TFIB entry");
+ if (params->seq < tfib->seq) {
+ INFO("Ignored ACK with seq %d < %d", params->seq, tfib->seq);
return;
}
- PARCEventTimer *timer =
- (PARCEventTimer *)mapmeTFIB_Get(TFIB(fibEntry), conn_in_id);
- if (!timer) {
- INFO(mapme,
- "[MAP-Me] - Ignored ACK for prefix not having the Connection in "
- "TFIB entry. Possible duplicate ?");
- return;
- }
-
- /* Stop timer and remove entry from TFIB */
- parcEventTimer_Stop(timer);
- mapmeTFIB_Remove(TFIB(fibEntry), conn_in_id);
-
- INFO(mapme, "[MAP-Me] - Removing TFIB entry for ack on connection %d",
- conn_in_id);
+ nexthops_remove(&tfib->nexthops, ingress_id);
+ mapme_on_event(mapme, MAPME_EVENT_PH_DEL, entry, ingress_id);
/* We need to update the timestamp only for IU Acks, not for IN Acks */
if (params->type == UPDATE_ACK) {
- INFO(mapme, "[MAP-Me] - Updating LastAckedUpdate");
- TFIB(fibEntry)->lastAckedUpdate = forwarder_GetTicks(mapme->forwarder);
- }
-}
-
-/*-----------------------------------------------------------------------------
- * Overloaded functions
- *----------------------------------------------------------------------------*/
-
-/*
- * @abstract returns where to forward a normal interests(nexthops) defined by
- * mapme, it also set the sequnence number properly if needed
- */
-
-/******************************************************************************
- * Public functions (exposed in the .h)
- ******************************************************************************/
-
-/*
- * Returns true iif the message corresponds to a MAP-Me packet
- */
-bool mapme_isMapMe(const uint8_t *packet) {
- hicn_mapme_header_t * mapme = (hicn_mapme_header_t*)packet;
-
- switch(HICN_IP_VERSION(packet)) {
- case 4:
- if (mapme->v4.ip.protocol != IPPROTO_ICMP)
- return false;
- return HICN_IS_MAPME(mapme->v4.icmp_rd.type, mapme->v4.icmp_rd.code);
- case 6:
- if (mapme->v6.ip.nxt != IPPROTO_ICMPV6)
- return false;
- return HICN_IS_MAPME(mapme->v6.icmp_rd.type, mapme->v6.icmp_rd.code);
- default:
- return false;
+ INFO(" - Updating LastAckedUpdate");
+ tfib->last_acked_update = ticks_now();
}
}
@@ -990,25 +1045,70 @@ bool mapme_isMapMe(const uint8_t *packet) {
* MAP-Me (eg. ICMP packets) and return higher level messages that can be
* processed by MAP-Me core.
*/
-void mapme_Process(const MapMe *mapme, const uint8_t *msgBuffer,
- unsigned conn_id) {
+void mapme_process(mapme_t *mapme, msgbuf_t *msgbuf) {
+ if (mapme->enabled == false) {
+ WARN("MAP-Me is NOT enabled");
+ return;
+ }
+
hicn_prefix_t prefix;
mapme_params_t params;
- hicn_mapme_parse_packet(msgBuffer, &prefix, &params);
+ uint8_t *packet = msgbuf_get_packet(msgbuf);
+ unsigned conn_id = msgbuf_get_connection_id(msgbuf);
+
+ int rc = hicn_mapme_parse_packet(packet, &prefix, &params);
+ if (rc < 0) return;
+
+ // XXX TYPE STR
+ INFO("Received interest type: %d seq: %d len:%d", params.type, params.seq,
+ prefix.len);
+
+ // XXX RENAME TYPES
switch (params.type) {
case UPDATE:
case NOTIFICATION:
- mapme_onSpecialInterest(mapme, msgBuffer, conn_id, &prefix, &params);
+ mapme_on_interest(mapme, msgbuf, conn_id, &prefix, &params);
break;
case UPDATE_ACK:
case NOTIFICATION_ACK:
- mapme_onSpecialInterestAck(mapme, msgBuffer, conn_id, &prefix, &params);
+ mapme_on_data(mapme, msgbuf, conn_id, &prefix, &params);
break;
default:
- ERR(mapme, "[MAP-Me] Unknown message");
+ ERROR("Unknown message");
break;
}
}
+#if 0
+/*
+ * Returns true iif the message corresponds to a MAP-Me packet
+ */
+bool mapme_match_packet(const uint8_t *packet) {
+ hicn_mapme_header_t *mapme = (hicn_mapme_header_t *)packet;
+
+ switch (HICN_IP_VERSION(packet)) {
+ case 4:
+ if (mapme->v4.ip.protocol != IPPROTO_ICMP) return false;
+ return HICN_IS_MAPME(mapme->v4.icmp_rd.type, mapme->v4.icmp_rd.code);
+ case 6:
+ if (mapme->v6.ip.nxt != IPPROTO_ICMPV6) return false;
+ return HICN_IS_MAPME(mapme->v6.icmp_rd.type, mapme->v6.icmp_rd.code);
+ default:
+ return false;
+ }
+}
+#endif
+
+void mapme_set_enable(mapme_t *mapme, bool enable) { mapme->enabled = enable; }
+void mapme_set_discovery(mapme_t *mapme, bool enable) {
+ mapme->discovery = enable;
+}
+void mapme_set_timescale(mapme_t *mapme, uint32_t time) {
+ mapme->timescale = time;
+}
+void mapme_set_retransmision(mapme_t *mapme, uint32_t time) {
+ mapme->retx = time;
+}
+
#endif /* WITH_MAPME */