[ovs-dev] [PATCH ovn v4 4/9] ovn-controller: Implement translation of OVN flows into OpenFlow.

Ben Pfaff blp at nicira.com
Wed Apr 29 17:12:30 UTC 2015


Signed-off-by: Ben Pfaff <blp at nicira.com>
---
 ovn/controller/automake.mk      |   4 +-
 ovn/controller/ovn-controller.c |   4 +
 ovn/controller/pipeline.c       | 373 ++++++++++++++++++++++++++++++++++++++++
 ovn/controller/pipeline.h       |  26 +++
 ovn/ovn-sb.ovsschema            |   2 +-
 5 files changed, 407 insertions(+), 2 deletions(-)
 create mode 100644 ovn/controller/pipeline.c
 create mode 100644 ovn/controller/pipeline.h

diff --git a/ovn/controller/automake.mk b/ovn/controller/automake.mk
index 4a266da..51c73be 100644
--- a/ovn/controller/automake.mk
+++ b/ovn/controller/automake.mk
@@ -5,7 +5,9 @@ ovn_controller_ovn_controller_SOURCES = \
 	ovn/controller/chassis.c \
 	ovn/controller/chassis.h \
 	ovn/controller/ovn-controller.c \
-	ovn/controller/ovn-controller.h
+	ovn/controller/ovn-controller.h \
+	ovn/controller/pipeline.c \
+	ovn/controller/pipeline.h
 ovn_controller_ovn_controller_LDADD = ovn/lib/libovn.la lib/libopenvswitch.la
 man_MANS += ovn/controller/ovn-controller.8
 EXTRA_DIST += ovn/controller/ovn-controller.8.xml
diff --git a/ovn/controller/ovn-controller.c b/ovn/controller/ovn-controller.c
index 44a4d5e..cfc562c 100644
--- a/ovn/controller/ovn-controller.c
+++ b/ovn/controller/ovn-controller.c
@@ -40,6 +40,7 @@
 #include "ovn-controller.h"
 #include "bindings.h"
 #include "chassis.h"
+#include "pipeline.h"
 
 VLOG_DEFINE_THIS_MODULE(main);
 
@@ -153,6 +154,7 @@ main(int argc, char *argv[])
 
     ctx.ovnsb_idl = ovsdb_idl_create(ovnsb_remote, &sbrec_idl_class,
                                      true, true);
+    pipeline_init(&ctx);
 
     get_initial_snapshot(ctx.ovnsb_idl);
 
@@ -179,6 +181,7 @@ main(int argc, char *argv[])
 
         chassis_run(&ctx);
         bindings_run(&ctx);
+        pipeline_run(&ctx);
         unixctl_server_run(unixctl);
 
         unixctl_server_wait(unixctl);
@@ -192,6 +195,7 @@ main(int argc, char *argv[])
     }
 
     unixctl_server_destroy(unixctl);
+    pipeline_destroy(&ctx);
     bindings_destroy(&ctx);
     chassis_destroy(&ctx);
 
diff --git a/ovn/controller/pipeline.c b/ovn/controller/pipeline.c
new file mode 100644
index 0000000..f124354
--- /dev/null
+++ b/ovn/controller/pipeline.c
@@ -0,0 +1,373 @@
+/* Copyright (c) 2015 Nicira, Inc.
+ *
+ * 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.
+ */
+
+#include <config.h>
+#include "pipeline.h"
+#include "dynamic-string.h"
+#include "ofp-actions.h"
+#include "ofpbuf.h"
+#include "openvswitch/vlog.h"
+#include "ovn/controller/ovn-controller.h"
+#include "ovn/lib/actions.h"
+#include "ovn/lib/expr.h"
+#include "ovn/lib/ovn-sb-idl.h"
+#include "simap.h"
+
+VLOG_DEFINE_THIS_MODULE(pipeline);
+
+/* Symbol table. */
+
+/* Contains "struct expr_symbol"s for fields supported by OVN pipeline. */
+static struct shash symtab;
+
+static void
+symtab_init(void)
+{
+    shash_init(&symtab);
+
+    /* Reserve a pair of registers for the logical inport and outport.  A full
+     * 32-bit register each is bigger than we need, but the expression code
+     * doesn't yet support string fields that occupy less than a full OXM. */
+    expr_symtab_add_string(&symtab, "inport", MFF_REG6, NULL);
+    expr_symtab_add_string(&symtab, "outport", MFF_REG7, NULL);
+
+    /* Registers.  We omit the registers that would otherwise overlap 'inport'
+     * and 'outport'. */
+    expr_symtab_add_field(&symtab, "xreg0", MFF_XREG0, NULL, false);
+    expr_symtab_add_field(&symtab, "xreg1", MFF_XREG1, NULL, false);
+    expr_symtab_add_field(&symtab, "xreg2", MFF_XREG2, NULL, false);
+
+    expr_symtab_add_subfield(&symtab, "reg0", NULL, "xreg0[32..63]");
+    expr_symtab_add_subfield(&symtab, "reg1", NULL, "xreg0[0..31]");
+    expr_symtab_add_subfield(&symtab, "reg2", NULL, "xreg1[32..63]");
+    expr_symtab_add_subfield(&symtab, "reg3", NULL, "xreg1[0..31]");
+    expr_symtab_add_subfield(&symtab, "reg4", NULL, "xreg2[32..63]");
+    expr_symtab_add_subfield(&symtab, "reg5", NULL, "xreg2[0..31]");
+
+    /* Data fields. */
+
+    expr_symtab_add_field(&symtab, "eth.src", MFF_ETH_SRC, NULL, false);
+    expr_symtab_add_field(&symtab, "eth.dst", MFF_ETH_DST, NULL, false);
+    expr_symtab_add_field(&symtab, "eth.type", MFF_ETH_TYPE, NULL, true);
+
+    expr_symtab_add_field(&symtab, "vlan.tci", MFF_VLAN_TCI, NULL, false);
+    expr_symtab_add_predicate(&symtab, "vlan.present", "vlan.tci[12]");
+    expr_symtab_add_subfield(&symtab, "vlan.pcp", "vlan.present",
+                             "vlan.tci[13..15]");
+    expr_symtab_add_subfield(&symtab, "vlan.vid", "vlan.present",
+                             "vlan.tci[0..11]");
+
+    expr_symtab_add_predicate(&symtab, "ip4", "eth.type == 0x800");
+    expr_symtab_add_predicate(&symtab, "ip6", "eth.type == 0x86dd");
+    expr_symtab_add_predicate(&symtab, "ip", "ip4 || ip6");
+    expr_symtab_add_field(&symtab, "ip.proto", MFF_IP_PROTO, "ip", true);
+    expr_symtab_add_field(&symtab, "ip.dscp", MFF_IP_DSCP, "ip", false);
+    expr_symtab_add_field(&symtab, "ip.ecn", MFF_IP_ECN, "ip", false);
+    expr_symtab_add_field(&symtab, "ip.ttl", MFF_IP_TTL, "ip", false);
+
+    expr_symtab_add_field(&symtab, "ip4.src", MFF_IPV4_SRC, "ip4", false);
+    expr_symtab_add_field(&symtab, "ip4.dst", MFF_IPV4_DST, "ip4", false);
+
+    expr_symtab_add_predicate(&symtab, "icmp4", "ip4 && ip.proto == 1");
+    expr_symtab_add_field(&symtab, "icmp4.type", MFF_ICMPV4_TYPE, "icmp4",
+              false);
+    expr_symtab_add_field(&symtab, "icmp4.code", MFF_ICMPV4_CODE, "icmp4",
+              false);
+
+    expr_symtab_add_field(&symtab, "ip6.src", MFF_IPV6_SRC, "ip6", false);
+    expr_symtab_add_field(&symtab, "ip6.dst", MFF_IPV6_DST, "ip6", false);
+    expr_symtab_add_field(&symtab, "ip6.label", MFF_IPV6_LABEL, "ip6", false);
+
+    expr_symtab_add_predicate(&symtab, "icmp6", "ip6 && ip.proto == 58");
+    expr_symtab_add_field(&symtab, "icmp6.type", MFF_ICMPV6_TYPE, "icmp6",
+                          true);
+    expr_symtab_add_field(&symtab, "icmp6.code", MFF_ICMPV6_CODE, "icmp6",
+                          true);
+
+    expr_symtab_add_predicate(&symtab, "icmp", "icmp4 || icmp6");
+
+    expr_symtab_add_field(&symtab, "ip.frag", MFF_IP_FRAG, "ip", false);
+    expr_symtab_add_predicate(&symtab, "ip.is_frag", "ip.frag[0]");
+    expr_symtab_add_predicate(&symtab, "ip.later_frag", "ip.frag[1]");
+    expr_symtab_add_predicate(&symtab, "ip.first_frag",
+                              "ip.is_frag && !ip.later_frag");
+
+    expr_symtab_add_predicate(&symtab, "arp", "eth.type == 0x806");
+    expr_symtab_add_field(&symtab, "arp.op", MFF_ARP_OP, "arp", false);
+    expr_symtab_add_field(&symtab, "arp.spa", MFF_ARP_SPA, "arp", false);
+    expr_symtab_add_field(&symtab, "arp.sha", MFF_ARP_SHA, "arp", false);
+    expr_symtab_add_field(&symtab, "arp.tpa", MFF_ARP_TPA, "arp", false);
+    expr_symtab_add_field(&symtab, "arp.tha", MFF_ARP_THA, "arp", false);
+
+    expr_symtab_add_predicate(&symtab, "nd",
+                              "icmp6.type == {135, 136} && icmp6.code == 0");
+    expr_symtab_add_field(&symtab, "nd.target", MFF_ND_TARGET, "nd", false);
+    expr_symtab_add_field(&symtab, "nd.sll", MFF_ND_SLL,
+              "nd && icmp6.type == 135", false);
+    expr_symtab_add_field(&symtab, "nd.tll", MFF_ND_TLL,
+              "nd && icmp6.type == 136", false);
+
+    expr_symtab_add_predicate(&symtab, "tcp", "ip.proto == 6");
+    expr_symtab_add_field(&symtab, "tcp.src", MFF_TCP_SRC, "tcp", false);
+    expr_symtab_add_field(&symtab, "tcp.dst", MFF_TCP_DST, "tcp", false);
+    expr_symtab_add_field(&symtab, "tcp.flags", MFF_TCP_FLAGS, "tcp", false);
+
+    expr_symtab_add_predicate(&symtab, "udp", "ip.proto == 17");
+    expr_symtab_add_field(&symtab, "udp.src", MFF_UDP_SRC, "udp", false);
+    expr_symtab_add_field(&symtab, "udp.dst", MFF_UDP_DST, "udp", false);
+
+    expr_symtab_add_predicate(&symtab, "sctp", "ip.proto == 132");
+    expr_symtab_add_field(&symtab, "sctp.src", MFF_SCTP_SRC, "sctp", false);
+    expr_symtab_add_field(&symtab, "sctp.dst", MFF_SCTP_DST, "sctp", false);
+}
+
+/* Logical datapaths and logical port numbers. */
+
+/* A logical datapath.
+ *
+ * 'uuid' is the UUID that represents the logical datapath in the OVN_SB
+ * database.
+ *
+ * 'integer' represents the logical datapath as an integer value that is unique
+ * only within the local hypervisor.  Because of its size, this value is more
+ * practical for use in an OpenFlow flow table than a UUID.
+ *
+ * 'ports' maps 'logical_port' names to 'tunnel_key' values in the OVN_SB
+ * Bindings table within the logical datapath. */
+struct logical_datapath {
+    struct hmap_node hmap_node; /* Indexed on 'uuid'. */
+    struct uuid uuid;           /* The logical_datapath's UUID. */
+    uint32_t integer;           /* Locally unique among logical datapaths. */
+    struct simap ports;         /* Logical port name to port number. */
+};
+
+/* Contains "struct logical_datapath"s. */
+static struct hmap logical_datapaths = HMAP_INITIALIZER(&logical_datapaths);
+
+/* Finds and returns the logical_datapath with the given 'uuid', or NULL if
+ * no such logical_datapath exists. */
+static struct logical_datapath *
+ldp_lookup(const struct uuid *uuid)
+{
+    struct logical_datapath *ldp;
+    HMAP_FOR_EACH_IN_BUCKET (ldp, hmap_node, uuid_hash(uuid),
+                             &logical_datapaths) {
+        if (uuid_equals(&ldp->uuid, uuid)) {
+            return ldp;
+        }
+    }
+    return NULL;
+}
+
+/* Creates a new logical_datapath with the given 'uuid'. */
+static struct logical_datapath *
+ldp_create(const struct uuid *uuid)
+{
+    static uint32_t next_integer = 1;
+    struct logical_datapath *ldp;
+
+    ldp = xmalloc(sizeof *ldp);
+    hmap_insert(&logical_datapaths, &ldp->hmap_node, uuid_hash(uuid));
+    ldp->uuid = *uuid;
+    ldp->integer = next_integer++;
+    simap_init(&ldp->ports);
+    return ldp;
+}
+
+/* Iterates through all of the records in the Bindings table, updating the
+ * table of logical_datapaths to match the values found in active Bindings. */
+static void
+ldp_run(struct controller_ctx *ctx)
+{
+    struct logical_datapath *ldp;
+    HMAP_FOR_EACH (ldp, hmap_node, &logical_datapaths) {
+        simap_clear(&ldp->ports);
+    }
+
+    const struct sbrec_bindings *binding;
+    SBREC_BINDINGS_FOR_EACH (binding, ctx->ovnsb_idl) {
+        struct logical_datapath *ldp;
+
+        ldp = ldp_lookup(&binding->logical_datapath);
+        if (!ldp) {
+            ldp = ldp_create(&binding->logical_datapath);
+        }
+
+        simap_put(&ldp->ports, binding->logical_port, binding->tunnel_key);
+    }
+
+    struct logical_datapath *next_ldp;
+    HMAP_FOR_EACH_SAFE (ldp, next_ldp, hmap_node, &logical_datapaths) {
+        if (simap_is_empty(&ldp->ports)) {
+            simap_destroy(&ldp->ports);
+            hmap_remove(&logical_datapaths, &ldp->hmap_node);
+            free(ldp);
+        }
+    }
+}
+
+static void
+ldp_destroy(void)
+{
+    struct logical_datapath *ldp, *next_ldp;
+    HMAP_FOR_EACH_SAFE (ldp, next_ldp, hmap_node, &logical_datapaths) {
+        simap_destroy(&ldp->ports);
+        hmap_remove(&logical_datapaths, &ldp->hmap_node);
+        free(ldp);
+    }
+}
+
+void
+pipeline_init(struct controller_ctx *ctx)
+{
+    symtab_init();
+
+    ovsdb_idl_add_column(ctx->ovnsb_idl, &sbrec_pipeline_col_logical_datapath);
+    ovsdb_idl_add_column(ctx->ovnsb_idl, &sbrec_pipeline_col_table_id);
+    ovsdb_idl_add_column(ctx->ovnsb_idl, &sbrec_pipeline_col_priority);
+    ovsdb_idl_add_column(ctx->ovnsb_idl, &sbrec_pipeline_col_match);
+    ovsdb_idl_add_column(ctx->ovnsb_idl, &sbrec_pipeline_col_actions);
+}
+
+static void
+add_ovn_flow(uint8_t table_id, uint16_t priority, const struct match *match,
+             const struct ofpbuf *ofpacts)
+{
+    struct ds s = DS_EMPTY_INITIALIZER;
+    ds_put_format(&s, "table_id=%"PRIu8", ", table_id);
+    ds_put_format(&s, "priority=%"PRIu16", ", priority);
+    match_format(match, &s, OFP_DEFAULT_PRIORITY);
+    ds_put_cstr(&s, ", actions=");
+    ofpacts_format(ofpacts->data, ofpacts->size, &s);
+    VLOG_INFO("%s", ds_cstr(&s));
+    ds_destroy(&s);
+}
+
+/* Translates logical flows in the Pipeline table in the OVN_SB database
+ * into OpenFlow flows. */
+void
+pipeline_run(struct controller_ctx *ctx)
+{
+    struct hmap flows = HMAP_INITIALIZER(&flows);
+    uint32_t conj_id_ofs = 1;
+
+    ldp_run(ctx);
+
+    VLOG_INFO("starting run...");
+    const struct sbrec_pipeline *pipeline;
+    SBREC_PIPELINE_FOR_EACH (pipeline, ctx->ovnsb_idl) {
+        /* Find the "struct logical_datapath" asssociated with this Pipeline
+         * row.  If there's no such struct, that must be because no logical
+         * ports are bound to that logical datapath, so there's no point in
+         * maintaining any flows for it anyway, so skip it. */
+        const struct logical_datapath *ldp;
+        ldp = ldp_lookup(&pipeline->logical_datapath);
+        if (!ldp) {
+            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
+            VLOG_INFO_RL(&rl,
+                         "logical flow for unknown logical datapath "UUID_FMT,
+                         UUID_ARGS(&pipeline->logical_datapath));
+            continue;
+        }
+
+        /* Translate OVN actions into OpenFlow actions. */
+        uint64_t ofpacts_stub[64 / 8];
+        struct ofpbuf ofpacts;
+        struct expr *prereqs;
+        char *error;
+
+        ofpbuf_use_stub(&ofpacts, ofpacts_stub, sizeof ofpacts_stub);
+        error = actions_parse_string(pipeline->actions, &symtab, &ldp->ports,
+                                     pipeline->table_id + 16,
+                                     &ofpacts, &prereqs);
+        if (error) {
+            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
+            VLOG_WARN_RL(&rl, "error parsing actions \"%s\": %s",
+                         pipeline->actions, error);
+            free(error);
+            continue;
+        }
+
+        /* Translate OVN match into table of OpenFlow matches. */
+        struct hmap matches;
+        struct expr *expr;
+
+        expr = expr_parse_string(pipeline->match, &symtab, &error);
+        if (!error) {
+            if (prereqs) {
+                expr = expr_combine(EXPR_T_AND, expr, prereqs);
+                prereqs = NULL;
+            }
+            expr = expr_annotate(expr, &symtab, &error);
+        }
+        if (error) {
+            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
+            VLOG_WARN_RL(&rl, "error parsing match \"%s\": %s",
+                         pipeline->match, error);
+            expr_destroy(prereqs);
+            ofpbuf_uninit(&ofpacts);
+            free(error);
+            continue;
+        }
+
+        expr = expr_simplify(expr);
+        expr = expr_normalize(expr);
+        uint32_t n_conjs = expr_to_matches(expr, &ldp->ports, &matches);
+        expr_destroy(expr);
+
+        /* Prepare the OpenFlow matches for adding to the flow table. */
+        struct expr_match *m;
+        HMAP_FOR_EACH (m, hmap_node, &matches) {
+            match_set_metadata(&m->match, htonll(ldp->integer));
+            if (m->match.wc.masks.conj_id) {
+                m->match.flow.conj_id += conj_id_ofs;
+            }
+            if (!m->n) {
+                add_ovn_flow(pipeline->table_id + 16, pipeline->priority,
+                             &m->match, &ofpacts);
+            } else {
+                uint64_t conj_stubs[64 / 8];
+                struct ofpbuf conj;
+
+                ofpbuf_use_stub(&conj, conj_stubs, sizeof conj_stubs);
+                for (int i = 0; i < m->n; i++) {
+                    const struct cls_conjunction *src = &m->conjunctions[i];
+                    struct ofpact_conjunction *dst;
+
+                    dst = ofpact_put_CONJUNCTION(&conj);
+                    dst->id = src->id + conj_id_ofs;
+                    dst->clause = src->clause;
+                    dst->n_clauses = src->n_clauses;
+                }
+                add_ovn_flow(pipeline->table_id + 16, pipeline->priority,
+                             &m->match, &conj);
+                ofpbuf_uninit(&conj);
+            }
+        }
+
+        /* Clean up. */
+        expr_matches_destroy(&matches);
+        ofpbuf_uninit(&ofpacts);
+        conj_id_ofs += n_conjs;
+    }
+    VLOG_INFO("...done");
+}
+
+void
+pipeline_destroy(struct controller_ctx *ctx OVS_UNUSED)
+{
+    expr_symtab_destroy(&symtab);
+    ldp_destroy();
+}
diff --git a/ovn/controller/pipeline.h b/ovn/controller/pipeline.h
new file mode 100644
index 0000000..d127bf5
--- /dev/null
+++ b/ovn/controller/pipeline.h
@@ -0,0 +1,26 @@
+/* Copyright (c) 2015 Nicira, Inc.
+ *
+ * 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.
+ */
+
+
+#ifndef OVN_PIPELINE_H
+#define OVN_PIPELINE_H 1
+
+struct controller_ctx;
+
+void pipeline_init(struct controller_ctx *);
+void pipeline_run(struct controller_ctx *);
+void pipeline_destroy(struct controller_ctx *);
+
+#endif /* ovn/pipeline.h */
diff --git a/ovn/ovn-sb.ovsschema b/ovn/ovn-sb.ovsschema
index 9fd5363..db56211 100644
--- a/ovn/ovn-sb.ovsschema
+++ b/ovn/ovn-sb.ovsschema
@@ -36,7 +36,7 @@
                 "logical_datapath": {"type": "uuid"},
                 "table_id": {"type": {"key": {"type": "integer",
                                               "minInteger": 0,
-                                              "maxInteger": 127}}},
+                                              "maxInteger": 31}}},
                 "priority": {"type": {"key": {"type": "integer",
                                               "minInteger": 0,
                                               "maxInteger": 65535}}},
-- 
2.1.3




More information about the dev mailing list