json: Move from lib to include/openvswitch.
[cascardo/ovs.git] / ofproto / ofproto-dpif-xlate.c
index 5272249..160da5b 100644 (file)
@@ -59,6 +59,7 @@
 #include "tnl-neigh-cache.h"
 #include "tnl-ports.h"
 #include "tunnel.h"
+#include "util.h"
 
 COVERAGE_DEFINE(xlate_actions);
 COVERAGE_DEFINE(xlate_actions_oversize);
@@ -191,9 +192,7 @@ struct xlate_ctx {
 
     /* Flow translation populates this with wildcards relevant in translation.
      * When 'xin->wc' is nonnull, this is the same pointer.  When 'xin->wc' is
-     * null, this is a pointer to uninitialized scratch memory.  This allows
-     * code to blindly write to 'ctx->wc' without worrying about whether the
-     * caller really wants wildcards. */
+     * null, this is a pointer to a temporary buffer. */
     struct flow_wildcards *wc;
 
     /* Output buffer for datapath actions.  When 'xin->odp_actions' is nonnull,
@@ -240,6 +239,7 @@ struct xlate_ctx {
     ofp_port_t nf_output_iface; /* Output interface index for NetFlow. */
     bool exit;                  /* No further actions should be processed. */
     mirror_mask_t mirrors;      /* Bitmap of associated mirrors. */
+    int mirror_snaplen;         /* Max size of a mirror packet in byte. */
 
    /* Freezing Translation
     * ====================
@@ -351,6 +351,12 @@ struct xlate_ctx {
     struct ofpbuf frozen_actions;
     const struct ofpact_controller *pause;
 
+    /* True if a packet was but is no longer MPLS (due to an MPLS pop action).
+     * This is a trigger for recirculation in cases where translating an action
+     * or looking up a flow requires access to the fields of the packet after
+     * the MPLS label stack that was originally present. */
+    bool was_mpls;
+
     /* True if conntrack has been performed on this packet during processing
      * on the current bridge. This is used to determine whether conntrack
      * state from the datapath should be honored after thawing. */
@@ -886,7 +892,7 @@ xlate_xport_copy(struct xbridge *xbridge, struct xbundle *xbundle,
  * This needs to be called after editing the xlate configuration.
  *
  * Functions that edit the new xlate configuration are
- * xlate_<ofport/bundle/ofport>_set and xlate_<ofport/bundle/ofport>_remove.
+ * xlate_<ofproto/bundle/ofport>_set and xlate_<ofproto/bundle/ofport>_remove.
  *
  * A sample workflow:
  *
@@ -1502,7 +1508,7 @@ group_is_alive(const struct xlate_ctx *ctx, uint32_t group_id, int depth)
 
         bucket = group_first_live_bucket(ctx, group, depth);
         group_dpif_unref(group);
-        return bucket == NULL;
+        return bucket != NULL;
     }
 
     return false;
@@ -1553,7 +1559,6 @@ group_best_live_bucket(const struct xlate_ctx *ctx,
 {
     struct ofputil_bucket *best_bucket = NULL;
     uint32_t best_score = 0;
-    int i = 0;
 
     struct ofputil_bucket *bucket;
     const struct ovs_list *buckets;
@@ -1561,13 +1566,13 @@ group_best_live_bucket(const struct xlate_ctx *ctx,
     group_dpif_get_buckets(group, &buckets);
     LIST_FOR_EACH (bucket, list_node, buckets) {
         if (bucket_is_alive(ctx, bucket, 0)) {
-            uint32_t score = (hash_int(i, basis) & 0xffff) * bucket->weight;
+            uint32_t score =
+                (hash_int(bucket->bucket_id, basis) & 0xffff) * bucket->weight;
             if (score >= best_score) {
                 best_bucket = bucket;
                 best_score = score;
             }
         }
-        i++;
     }
 
     return best_bucket;
@@ -1698,12 +1703,15 @@ mirror_packet(struct xlate_ctx *ctx, struct xbundle *xbundle,
         mirror_mask_t dup_mirrors;
         struct ofbundle *out;
         int out_vlan;
+        int snaplen;
 
         /* Get the details of the mirror represented by the rightmost 1-bit. */
         bool has_mirror = mirror_get(xbridge->mbridge, raw_ctz(mirrors),
-                                     &vlans, &dup_mirrors, &out, &out_vlan);
+                                     &vlans, &dup_mirrors,
+                                     &out, &snaplen, &out_vlan);
         ovs_assert(has_mirror);
 
+
         /* If this mirror selects on the basis of VLAN, and it does not select
          * 'vlan', then discard this mirror and go on to the next one. */
         if (vlans) {
@@ -1719,6 +1727,7 @@ mirror_packet(struct xlate_ctx *ctx, struct xbundle *xbundle,
          * done now to ensure that output_normal(), below, doesn't recursively
          * output to the same mirrors. */
         ctx->mirrors |= dup_mirrors;
+        ctx->mirror_snaplen = snaplen;
 
         /* Send the packet to the mirror. */
         if (out) {
@@ -1742,6 +1751,7 @@ mirror_packet(struct xlate_ctx *ctx, struct xbundle *xbundle,
         /* output_normal() could have recursively output (to different
          * mirrors), so make sure that we don't send duplicates. */
         mirrors &= ~ctx->mirrors;
+        ctx->mirror_snaplen = 0;
     }
 }
 
@@ -2390,6 +2400,20 @@ xlate_normal_flood(struct xlate_ctx *ctx, struct xbundle *in_xbundle,
     ctx->nf_output_iface = NF_OUT_FLOOD;
 }
 
+static bool
+is_ip_local_multicast(const struct flow *flow, struct flow_wildcards *wc)
+{
+    if (flow->dl_type == htons(ETH_TYPE_IP)) {
+        memset(&wc->masks.nw_dst, 0xff, sizeof wc->masks.nw_dst);
+        return ip_is_local_multicast(flow->nw_dst);
+    } else if (flow->dl_type == htons(ETH_TYPE_IPV6)) {
+        memset(&wc->masks.ipv6_dst, 0xff, sizeof wc->masks.ipv6_dst);
+        return ipv6_is_all_hosts(&flow->ipv6_dst);
+    } else {
+        return false;
+    }
+}
+
 static void
 xlate_normal(struct xlate_ctx *ctx)
 {
@@ -2473,7 +2497,8 @@ xlate_normal(struct xlate_ctx *ctx)
         struct mcast_snooping *ms = ctx->xbridge->ms;
         struct mcast_group *grp = NULL;
 
-        if (is_igmp(flow)) {
+        if (is_igmp(flow, wc)) {
+            memset(&wc->masks.tp_src, 0xff, sizeof wc->masks.tp_src);
             if (mcast_snooping_is_membership(flow->tp_src) ||
                 mcast_snooping_is_query(flow->tp_src)) {
                 if (ctx->xin->may_learn && ctx->xin->packet) {
@@ -2506,13 +2531,13 @@ xlate_normal(struct xlate_ctx *ctx)
                 xlate_normal_flood(ctx, in_xbundle, vlan);
             }
             return;
-        } else if (is_mld(flow)) {
+        } else if (is_mld(flow, wc)) {
             ctx->xout->slow |= SLOW_ACTION;
             if (ctx->xin->may_learn && ctx->xin->packet) {
                 update_mcast_snooping_table(ctx->xbridge, flow, vlan,
                                             in_xbundle, ctx->xin->packet);
             }
-            if (is_mld_report(flow)) {
+            if (is_mld_report(flow, wc)) {
                 ovs_rwlock_rdlock(&ms->rwlock);
                 xlate_normal_mcast_send_mrouters(ctx, ms, in_xbundle, vlan);
                 xlate_normal_mcast_send_rports(ctx, ms, in_xbundle, vlan);
@@ -2522,10 +2547,7 @@ xlate_normal(struct xlate_ctx *ctx)
                 xlate_normal_flood(ctx, in_xbundle, vlan);
             }
         } else {
-            if ((flow->dl_type == htons(ETH_TYPE_IP)
-                 && ip_is_local_multicast(flow->nw_dst))
-                || (flow->dl_type == htons(ETH_TYPE_IPV6)
-                    && ipv6_is_all_hosts(&flow->ipv6_dst))) {
+            if (is_ip_local_multicast(flow, wc)) {
                 /* RFC4541: section 2.1.2, item 2: Packets with a dst IP
                  * address in the 224.0.0.x range which are not IGMP must
                  * be forwarded on all ports */
@@ -2585,7 +2607,10 @@ xlate_normal(struct xlate_ctx *ctx)
  * 'cookie' (of length 'cookie_size' bytes) is passed back in the callback for
  * each sampled packet.  'tunnel_out_port', if not ODPP_NONE, is added as the
  * OVS_USERSPACE_ATTR_EGRESS_TUN_PORT attribute.  If 'include_actions', an
- * OVS_USERSPACE_ATTR_ACTIONS attribute is added.
+ * OVS_USERSPACE_ATTR_ACTIONS attribute is added.  If 'emit_set_tunnel',
+ * sample(sampling_port=1) would translate into datapath sample action
+ * set(tunnel(...)), sample(...) and it is used for sampling egress tunnel
+ * information.
  */
 static size_t
 compose_sample_action(struct xlate_ctx *ctx,
@@ -2639,8 +2664,10 @@ compose_sflow_action(struct xlate_ctx *ctx)
                                  true);
 }
 
-/* If IPFIX is enabled, this appends a "sample" action to implement IPFIX to
- * 'ctx->odp_actions'. */
+/* If flow IPFIX is enabled, make sure IPFIX flow sample action
+ * at egress point of tunnel port is just in front of corresponding
+ * output action. If bridge IPFIX is enabled, this appends an IPFIX
+ * sample action to 'ctx->odp_actions'. */
 static void
 compose_ipfix_action(struct xlate_ctx *ctx, odp_port_t output_odp_port)
 {
@@ -2658,7 +2685,7 @@ compose_ipfix_action(struct xlate_ctx *ctx, odp_port_t output_odp_port)
         return;
     }
 
-    /* For output case, output_odp_port is valid*/
+    /* For output case, output_odp_port is valid*/
     if (output_odp_port != ODPP_NONE) {
         if (!dpif_ipfix_get_bridge_exporter_output_sampling(ipfix)) {
             return;
@@ -2872,6 +2899,7 @@ static int
 build_tunnel_send(struct xlate_ctx *ctx, const struct xport *xport,
                   const struct flow *flow, odp_port_t tunnel_odp_port)
 {
+    struct netdev_tnl_build_header_params tnl_params;
     struct ovs_action_push_tnl tnl_push_data;
     struct xport *out_dev = NULL;
     ovs_be32 s_ip = 0, d_ip = 0;
@@ -2932,8 +2960,8 @@ build_tunnel_send(struct xlate_ctx *ctx, const struct xport *xport,
                  ETH_ADDR_ARGS(smac), ipv6_string_mapped(buf_sip6, &s_ip6),
                  ETH_ADDR_ARGS(dmac), buf_dip6);
 
-    err = tnl_port_build_header(xport->ofport, flow,
-                                dmac, smac, &s_ip6, &tnl_push_data);
+    netdev_init_tnl_build_header_params(&tnl_params, flow, &s_ip6, dmac, smac);
+    err = tnl_port_build_header(xport->ofport, &tnl_push_data, &tnl_params);
     if (err) {
         return err;
     }
@@ -2979,7 +3007,7 @@ compose_output_action__(struct xlate_ctx *ctx, ofp_port_t ofp_port,
 
     /* If 'struct flow' gets additional metadata, we'll need to zero it out
      * before traversing a patch port. */
-    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 35);
+    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 36);
     memset(&flow_tnl, 0, sizeof flow_tnl);
 
     if (!xport) {
@@ -2988,6 +3016,9 @@ compose_output_action__(struct xlate_ctx *ctx, ofp_port_t ofp_port,
     } else if (xport->config & OFPUTIL_PC_NO_FWD) {
         xlate_report(ctx, "OFPPC_NO_FWD set, skipping output");
         return;
+    } else if (ctx->mirror_snaplen != 0 && xport->odp_port == ODPP_NONE) {
+        xlate_report(ctx, "Mirror truncate to ODPP_NONE, skipping output");
+        return;
     } else if (check_stp) {
         if (is_stp(&ctx->base_flow)) {
             if (!xport_stp_should_forward_bpdu(xport) &&
@@ -3018,6 +3049,7 @@ compose_output_action__(struct xlate_ctx *ctx, ofp_port_t ofp_port,
         const struct xport *peer = xport->peer;
         struct flow old_flow = ctx->xin->flow;
         bool old_conntrack = ctx->conntracked;
+        bool old_was_mpls = ctx->was_mpls;
         cls_version_t old_version = ctx->tables_version;
         struct ofpbuf old_stack = ctx->stack;
         union mf_subvalue new_stack[1024 / sizeof(union mf_subvalue)];
@@ -3075,6 +3107,10 @@ compose_output_action__(struct xlate_ctx *ctx, ofp_port_t ofp_port,
         /* Restore calling bridge's lookup version. */
         ctx->tables_version = old_version;
 
+        /* The peer bridge popping MPLS should have no effect on the original
+         * bridge. */
+        ctx->was_mpls = old_was_mpls;
+
         /* The peer bridge's conntrack execution should have no effect on the
          * original bridge. */
         ctx->conntracked = old_conntrack;
@@ -3199,11 +3235,26 @@ compose_output_action__(struct xlate_ctx *ctx, ofp_port_t ofp_port,
                     /* Tunnel push-pop action is not compatible with
                      * IPFIX action. */
                     compose_ipfix_action(ctx, out_port);
+
+                    /* Handle truncation of the mirrored packet. */
+                    if (ctx->mirror_snaplen > 0 &&
+                        ctx->mirror_snaplen < UINT16_MAX) {
+                        struct ovs_action_trunc *trunc;
+
+                        trunc = nl_msg_put_unspec_uninit(ctx->odp_actions,
+                                                         OVS_ACTION_ATTR_TRUNC,
+                                                         sizeof *trunc);
+                        trunc->max_len = ctx->mirror_snaplen;
+                        if (!ctx->xbridge->support.trunc) {
+                            ctx->xout->slow |= SLOW_ACTION;
+                        }
+                    }
+
                     nl_msg_put_odp_port(ctx->odp_actions,
                                         OVS_ACTION_ATTR_OUTPUT,
                                         out_port);
-               }
-           }
+                }
+            }
         }
 
         ctx->sflow_odp_port = odp_port;
@@ -3283,6 +3334,11 @@ static void
 xlate_table_action(struct xlate_ctx *ctx, ofp_port_t in_port, uint8_t table_id,
                    bool may_packet_in, bool honor_table_miss)
 {
+    /* Check if we need to recirculate before matching in a table. */
+    if (ctx->was_mpls) {
+        ctx_trigger_freeze(ctx);
+        return;
+    }
     if (xlate_resubmit_resource_check(ctx)) {
         uint8_t old_table_id = ctx->table_id;
         struct rule_dpif *rule;
@@ -3291,7 +3347,7 @@ xlate_table_action(struct xlate_ctx *ctx, ofp_port_t in_port, uint8_t table_id,
 
         rule = rule_dpif_lookup_from_table(ctx->xbridge->ofproto,
                                            ctx->tables_version,
-                                           &ctx->xin->flow, ctx->xin->wc,
+                                           &ctx->xin->flow, ctx->wc,
                                            ctx->xin->resubmit_stats,
                                            &ctx->table_id, in_port,
                                            may_packet_in, honor_table_miss);
@@ -3344,6 +3400,7 @@ xlate_group_bucket(struct xlate_ctx *ctx, struct ofputil_bucket *bucket)
     struct ofpbuf action_set = ofpbuf_const_initializer(bucket->ofpacts,
                                                         bucket->ofpacts_len);
     struct flow old_flow = ctx->xin->flow;
+    bool old_was_mpls = ctx->was_mpls;
 
     ofpacts_execute_action_set(&action_list, &action_set);
     ctx->indentation++;
@@ -3375,6 +3432,10 @@ xlate_group_bucket(struct xlate_ctx *ctx, struct ofputil_bucket *bucket)
      * group buckets. */
     ctx->xin->flow = old_flow;
 
+    /* The group bucket popping MPLS should have no effect after bucket
+     * execution. */
+    ctx->was_mpls = old_was_mpls;
+
     /* The fact that the group bucket exits (for any reason) does not mean that
      * the translation after the group action should exit.  Specifically, if
      * the group bucket freezes translation, the actions after the group action
@@ -3498,6 +3559,13 @@ xlate_select_group(struct xlate_ctx *ctx, struct group_dpif *group)
 {
     const char *selection_method = group_dpif_get_selection_method(group);
 
+    /* Select groups may access flow keys beyond L2 in order to
+     * select a bucket. Recirculate as appropriate to make this possible.
+     */
+    if (ctx->was_mpls) {
+        ctx_trigger_freeze(ctx);
+    }
+
     if (selection_method[0] == '\0') {
         xlate_default_select_group(ctx, group);
     } else if (!strcasecmp("hash", selection_method)) {
@@ -3607,6 +3675,7 @@ execute_controller_action(struct xlate_ctx *ctx, int len,
                           uint16_t controller_id,
                           const uint8_t *userdata, size_t userdata_len)
 {
+    struct dp_packet_batch batch;
     struct dp_packet *packet;
 
     ctx->xout->slow |= SLOW_CONTROLLER;
@@ -3616,8 +3685,8 @@ execute_controller_action(struct xlate_ctx *ctx, int len,
     }
 
     packet = dp_packet_clone(ctx->xin->packet);
-
-    odp_execute_actions(NULL, &packet, 1, false,
+    packet_batch_init_packet(&batch, packet);
+    odp_execute_actions(NULL, &batch, false,
                         ctx->odp_actions->data, ctx->odp_actions->size, NULL);
 
     /* A packet sent by an action in a table-miss rule is considered an
@@ -3790,7 +3859,7 @@ compose_mpls_pop_action(struct xlate_ctx *ctx, ovs_be16 eth_type)
 
     if (flow_pop_mpls(flow, n, eth_type, ctx->wc)) {
         if (!eth_type_mpls(eth_type) && ctx->xbridge->support.odp.recirc) {
-            ctx_trigger_freeze(ctx);
+            ctx->was_mpls = true;
         }
     } else if (n >= FLOW_MAX_MPLS_LABELS) {
         if (ctx->xin->packet != NULL) {
@@ -3949,6 +4018,56 @@ xlate_output_reg_action(struct xlate_ctx *ctx,
     }
 }
 
+static void
+xlate_output_trunc_action(struct xlate_ctx *ctx,
+                    ofp_port_t port, uint32_t max_len)
+{
+    bool support_trunc = ctx->xbridge->support.trunc;
+    struct ovs_action_trunc *trunc;
+    char name[OFP_MAX_PORT_NAME_LEN];
+
+    switch (port) {
+    case OFPP_TABLE:
+    case OFPP_NORMAL:
+    case OFPP_FLOOD:
+    case OFPP_ALL:
+    case OFPP_CONTROLLER:
+    case OFPP_NONE:
+        ofputil_port_to_string(port, name, sizeof name);
+        xlate_report(ctx, "output_trunc does not support port: %s", name);
+        break;
+    case OFPP_LOCAL:
+    case OFPP_IN_PORT:
+    default:
+        if (port != ctx->xin->flow.in_port.ofp_port) {
+            const struct xport *xport = get_ofp_port(ctx->xbridge, port);
+
+            if (xport == NULL || xport->odp_port == ODPP_NONE) {
+                /* Since truncate happens at its following output action, if
+                 * the output port is a patch port, the behavior is somehow
+                 * unpredicable. For simpilicity, disallow this case. */
+                ofputil_port_to_string(port, name, sizeof name);
+                XLATE_REPORT_ERROR(ctx, "bridge %s: "
+                         "output_trunc does not support port: %s",
+                         ctx->xbridge->name, name);
+                break;
+            }
+
+            trunc = nl_msg_put_unspec_uninit(ctx->odp_actions,
+                                OVS_ACTION_ATTR_TRUNC,
+                                sizeof *trunc);
+            trunc->max_len = max_len;
+            xlate_output_action(ctx, port, max_len, false);
+            if (!support_trunc) {
+                ctx->xout->slow |= SLOW_ACTION;
+            }
+        } else {
+            xlate_report(ctx, "skipping output to input port");
+        }
+        break;
+    }
+}
+
 static void
 xlate_enqueue_action(struct xlate_ctx *ctx,
                      const struct ofpact_enqueue *enqueue)
@@ -4105,6 +4224,15 @@ static void
 xlate_sample_action(struct xlate_ctx *ctx,
                     const struct ofpact_sample *os)
 {
+    odp_port_t output_odp_port = ODPP_NONE;
+    odp_port_t tunnel_out_port = ODPP_NONE;
+    struct dpif_ipfix *ipfix = ctx->xbridge->ipfix;
+    bool emit_set_tunnel = false;
+
+    if (!ipfix || ctx->xin->flow.in_port.ofp_port == OFPP_NONE) {
+        return;
+    }
+
     /* Scale the probability from 16-bit to 32-bit while representing
      * the same percentage. */
     uint32_t probability = (os->probability << 16) | os->probability;
@@ -4118,7 +4246,51 @@ xlate_sample_action(struct xlate_ctx *ctx,
         return;
     }
 
-    xlate_commit_actions(ctx);
+    /* If ofp_port in flow sample action is equel to ofp_port,
+     * this sample action is a input port action. */
+    if (os->sampling_port != OFPP_NONE &&
+        os->sampling_port != ctx->xin->flow.in_port.ofp_port) {
+        output_odp_port = ofp_port_to_odp_port(ctx->xbridge,
+                                               os->sampling_port);
+        if (output_odp_port == ODPP_NONE) {
+            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
+            VLOG_WARN_RL(&rl, "can't use unknown port %d in flow sample "
+                         "action", os->sampling_port);
+            return;
+        }
+
+        if (dpif_ipfix_get_flow_exporter_tunnel_sampling(ipfix,
+                                                         os->collector_set_id)
+            && dpif_ipfix_get_tunnel_port(ipfix, output_odp_port)) {
+            tunnel_out_port = output_odp_port;
+            emit_set_tunnel = true;
+        }
+    }
+
+     xlate_commit_actions(ctx);
+    /* If 'emit_set_tunnel', sample(sampling_port=1) would translate
+     * into datapath sample action set(tunnel(...)), sample(...) and
+     * it is used for sampling egress tunnel information. */
+    if (emit_set_tunnel) {
+        const struct xport *xport = get_ofp_port(ctx->xbridge,
+                                                 os->sampling_port);
+
+        if (xport && xport->is_tunnel) {
+            struct flow *flow = &ctx->xin->flow;
+            tnl_port_send(xport->ofport, flow, ctx->wc);
+            if (!ovs_native_tunneling_is_on(ctx->xbridge->ofproto)) {
+                struct flow_tnl flow_tnl = flow->tunnel;
+
+                commit_odp_tunnel_action(flow, &ctx->base_flow,
+                                         ctx->odp_actions);
+                flow->tunnel = flow_tnl;
+            }
+        } else {
+            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
+            VLOG_WARN_RL(&rl, "sampling_port:%d should be a tunnel port.",
+                         os->sampling_port);
+        }
+    }
 
     union user_action_cookie cookie = {
         .flow_sample = {
@@ -4127,10 +4299,11 @@ xlate_sample_action(struct xlate_ctx *ctx,
             .collector_set_id = os->collector_set_id,
             .obs_domain_id = os->obs_domain_id,
             .obs_point_id = os->obs_point_id,
+            .output_odp_port = output_odp_port,
         }
     };
     compose_sample_action(ctx, probability, &cookie, sizeof cookie.flow_sample,
-                          ODPP_NONE, false);
+                          tunnel_out_port, false);
 }
 
 static bool
@@ -4234,6 +4407,7 @@ freeze_unroll_actions(const struct ofpact *a, const struct ofpact *end,
     for (; a < end; a = ofpact_next(a)) {
         switch (a->type) {
         case OFPACT_OUTPUT_REG:
+        case OFPACT_OUTPUT_TRUNC:
         case OFPACT_GROUP:
         case OFPACT_OUTPUT:
         case OFPACT_CONTROLLER:
@@ -4327,7 +4501,7 @@ static void
 put_ct_label(const struct flow *flow, struct ofpbuf *odp_actions,
              struct flow_wildcards *wc)
 {
-    if (!ovs_u128_is_zero(&wc->masks.ct_label)) {
+    if (!ovs_u128_is_zero(wc->masks.ct_label)) {
         struct {
             ovs_u128 key;
             ovs_u128 mask;
@@ -4465,6 +4639,96 @@ compose_conntrack_action(struct xlate_ctx *ctx, struct ofpact_conntrack *ofc)
     }
 }
 
+static void
+recirc_for_mpls(const struct ofpact *a, struct xlate_ctx *ctx)
+{
+    /* No need to recirculate if already exiting. */
+    if (ctx->exit) {
+        return;
+    }
+
+    /* Do not consider recirculating unless the packet was previously MPLS. */
+    if (!ctx->was_mpls) {
+        return;
+    }
+
+    /* Special case these actions, only recirculating if necessary.
+     * This avoids the overhead of recirculation in common use-cases.
+     */
+    switch (a->type) {
+
+    /* Output actions  do not require recirculation. */
+    case OFPACT_OUTPUT:
+    case OFPACT_OUTPUT_TRUNC:
+    case OFPACT_ENQUEUE:
+    case OFPACT_OUTPUT_REG:
+    /* Set actions that don't touch L3+ fields do not require recirculation. */
+    case OFPACT_SET_VLAN_VID:
+    case OFPACT_SET_VLAN_PCP:
+    case OFPACT_SET_ETH_SRC:
+    case OFPACT_SET_ETH_DST:
+    case OFPACT_SET_TUNNEL:
+    case OFPACT_SET_QUEUE:
+    /* If actions of a group require recirculation that can be detected
+     * when translating them. */
+    case OFPACT_GROUP:
+        return;
+
+    /* Set field that don't touch L3+ fields don't require recirculation. */
+    case OFPACT_SET_FIELD:
+        if (mf_is_l3_or_higher(ofpact_get_SET_FIELD(a)->field)) {
+            break;
+        }
+        return;
+
+    /* For simplicity, recirculate in all other cases. */
+    case OFPACT_CONTROLLER:
+    case OFPACT_BUNDLE:
+    case OFPACT_STRIP_VLAN:
+    case OFPACT_PUSH_VLAN:
+    case OFPACT_SET_IPV4_SRC:
+    case OFPACT_SET_IPV4_DST:
+    case OFPACT_SET_IP_DSCP:
+    case OFPACT_SET_IP_ECN:
+    case OFPACT_SET_IP_TTL:
+    case OFPACT_SET_L4_SRC_PORT:
+    case OFPACT_SET_L4_DST_PORT:
+    case OFPACT_REG_MOVE:
+    case OFPACT_STACK_PUSH:
+    case OFPACT_STACK_POP:
+    case OFPACT_DEC_TTL:
+    case OFPACT_SET_MPLS_LABEL:
+    case OFPACT_SET_MPLS_TC:
+    case OFPACT_SET_MPLS_TTL:
+    case OFPACT_DEC_MPLS_TTL:
+    case OFPACT_PUSH_MPLS:
+    case OFPACT_POP_MPLS:
+    case OFPACT_POP_QUEUE:
+    case OFPACT_FIN_TIMEOUT:
+    case OFPACT_RESUBMIT:
+    case OFPACT_LEARN:
+    case OFPACT_CONJUNCTION:
+    case OFPACT_MULTIPATH:
+    case OFPACT_NOTE:
+    case OFPACT_EXIT:
+    case OFPACT_SAMPLE:
+    case OFPACT_UNROLL_XLATE:
+    case OFPACT_CT:
+    case OFPACT_NAT:
+    case OFPACT_DEBUG_RECIRC:
+    case OFPACT_METER:
+    case OFPACT_CLEAR_ACTIONS:
+    case OFPACT_WRITE_ACTIONS:
+    case OFPACT_WRITE_METADATA:
+    case OFPACT_GOTO_TABLE:
+    default:
+        break;
+    }
+
+    /* Recirculate */
+    ctx_trigger_freeze(ctx);
+}
+
 static void
 do_xlate_actions(const struct ofpact *ofpacts, size_t ofpacts_len,
                  struct xlate_ctx *ctx)
@@ -4488,6 +4752,8 @@ do_xlate_actions(const struct ofpact *ofpacts, size_t ofpacts_len,
             break;
         }
 
+        recirc_for_mpls(a, ctx);
+
         if (ctx->exit) {
             /* Check if need to store the remaining actions for later
              * execution. */
@@ -4743,6 +5009,11 @@ do_xlate_actions(const struct ofpact *ofpacts, size_t ofpacts_len,
             xlate_output_reg_action(ctx, ofpact_get_OUTPUT_REG(a));
             break;
 
+        case OFPACT_OUTPUT_TRUNC:
+            xlate_output_trunc_action(ctx, ofpact_get_OUTPUT_TRUNC(a)->port,
+                                ofpact_get_OUTPUT_TRUNC(a)->max_len);
+            break;
+
         case OFPACT_LEARN:
             xlate_learn_action(ctx, ofpact_get_LEARN(a));
             break;
@@ -5067,7 +5338,7 @@ xlate_wc_finish(struct xlate_ctx *ctx)
      * Avoid the problem here by making sure that only the low 8 bits of
      * either field can be unwildcarded for ICMP.
      */
-    if (is_icmpv4(&ctx->xin->flow) || is_icmpv6(&ctx->xin->flow)) {
+    if (is_icmpv4(&ctx->xin->flow, NULL) || is_icmpv6(&ctx->xin->flow, NULL)) {
         ctx->wc->masks.tp_src &= htons(UINT8_MAX);
         ctx->wc->masks.tp_dst &= htons(UINT8_MAX);
     }
@@ -5104,7 +5375,6 @@ xlate_actions(struct xlate_in *xin, struct xlate_out *xout)
     union mf_subvalue stack_stub[1024 / sizeof(union mf_subvalue)];
     uint64_t action_set_stub[1024 / 8];
     uint64_t frozen_actions_stub[1024 / 8];
-    struct flow_wildcards scratch_wc;
     uint64_t actions_stub[256 / 8];
     struct ofpbuf scratch_actions = OFPBUF_STUB_INITIALIZER(actions_stub);
     struct xlate_ctx ctx = {
@@ -5115,7 +5385,9 @@ xlate_actions(struct xlate_in *xin, struct xlate_out *xout)
         .xbridge = xbridge,
         .stack = OFPBUF_STUB_INITIALIZER(stack_stub),
         .rule = xin->rule,
-        .wc = xin->wc ? xin->wc : &scratch_wc,
+        .wc = (xin->wc
+               ? xin->wc
+               : &(struct flow_wildcards) { .masks = { .dl_type = 0 } }),
         .odp_actions = xin->odp_actions ? xin->odp_actions : &scratch_actions,
 
         .indentation = xin->indentation,
@@ -5138,6 +5410,7 @@ xlate_actions(struct xlate_in *xin, struct xlate_out *xout)
         .frozen_actions = OFPBUF_STUB_INITIALIZER(frozen_actions_stub),
         .pause = NULL,
 
+        .was_mpls = false,
         .conntracked = false,
 
         .ct_nat_action = NULL,
@@ -5155,9 +5428,7 @@ xlate_actions(struct xlate_in *xin, struct xlate_out *xout)
     memset(&ctx.base_flow.tunnel, 0, sizeof ctx.base_flow.tunnel);
 
     ofpbuf_reserve(ctx.odp_actions, NL_A_U32_SIZE);
-    if (xin->wc) {
-        xlate_wc_init(&ctx);
-    }
+    xlate_wc_init(&ctx);
 
     COVERAGE_INC(xlate_actions);
 
@@ -5247,7 +5518,7 @@ xlate_actions(struct xlate_in *xin, struct xlate_out *xout)
 
     if (!xin->ofpacts && !ctx.rule) {
         ctx.rule = rule_dpif_lookup_from_table(
-            ctx.xbridge->ofproto, ctx.tables_version, flow, xin->wc,
+            ctx.xbridge->ofproto, ctx.tables_version, flow, ctx.wc,
             ctx.xin->resubmit_stats, &ctx.table_id,
             flow->in_port.ofp_port, true, true);
         if (ctx.xin->resubmit_stats) {
@@ -5398,9 +5669,7 @@ xlate_actions(struct xlate_in *xin, struct xlate_out *xout)
         }
     }
 
-    if (xin->wc) {
-        xlate_wc_finish(&ctx);
-    }
+    xlate_wc_finish(&ctx);
 
 exit:
     ofpbuf_uninit(&ctx.stack);
@@ -5479,18 +5748,21 @@ xlate_resume(struct ofproto_dpif *ofproto,
     return error == XLATE_BRIDGE_NOT_FOUND ? OFPERR_NXR_STALE : 0;
 }
 
-/* Sends 'packet' out 'ofport'.
+/* Sends 'packet' out 'ofport'. If 'port' is a tunnel and that tunnel type
+ * supports a notion of an OAM flag, sets it if 'oam' is true.
  * May modify 'packet'.
  * Returns 0 if successful, otherwise a positive errno value. */
 int
-xlate_send_packet(const struct ofport_dpif *ofport, struct dp_packet *packet)
+xlate_send_packet(const struct ofport_dpif *ofport, bool oam,
+                  struct dp_packet *packet)
 {
     struct xlate_cfg *xcfg = ovsrcu_get(struct xlate_cfg *, &xcfgp);
     struct xport *xport;
-    struct ofpact_output output;
+    uint64_t ofpacts_stub[1024 / 8];
+    struct ofpbuf ofpacts;
     struct flow flow;
 
-    ofpact_init(&output.ofpact, OFPACT_OUTPUT, sizeof output);
+    ofpbuf_use_stack(&ofpacts, ofpacts_stub, sizeof ofpacts_stub);
     /* Use OFPP_NONE as the in_port to avoid special packet processing. */
     flow_extract(packet, &flow);
     flow.in_port.ofp_port = OFPP_NONE;
@@ -5499,12 +5771,19 @@ xlate_send_packet(const struct ofport_dpif *ofport, struct dp_packet *packet)
     if (!xport) {
         return EINVAL;
     }
-    output.port = xport->ofp_port;
-    output.max_len = 0;
+
+    if (oam) {
+        struct ofpact_set_field *sf = ofpact_put_SET_FIELD(&ofpacts);
+
+        sf->field = mf_from_id(MFF_TUN_FLAGS);
+        sf->value.be16 = htons(NX_TUN_FLAG_OAM);
+        sf->mask.be16 = htons(NX_TUN_FLAG_OAM);
+    }
+
+    ofpact_put_OUTPUT(&ofpacts)->port = xport->ofp_port;
 
     return ofproto_dpif_execute_actions(xport->xbridge->ofproto, &flow, NULL,
-                                        &output.ofpact, sizeof output,
-                                        packet);
+                                        ofpacts.data, ofpacts.size, packet);
 }
 
 struct xlate_cache *