ovn: Rename Pipeline table to Logical_Flow table.
authorBen Pfaff <blp@nicira.com>
Mon, 3 Aug 2015 18:10:43 +0000 (11:10 -0700)
committerBen Pfaff <blp@nicira.com>
Mon, 3 Aug 2015 18:38:35 +0000 (11:38 -0700)
The OVN pipeline is being split into two phases, which are most naturally
called "pipelines".  I kept getting very confused trying to call them
anything else, and in the end it seems to make more sense to just rename
the Pipeline table.

Signed-off-by: Ben Pfaff <blp@nicira.com>
Acked-by: Justin Pettit <jpettit@nicira.com>
14 files changed:
ovn/TODO
ovn/controller/automake.mk
ovn/controller/lflow.c [new file with mode: 0644]
ovn/controller/lflow.h [new file with mode: 0644]
ovn/controller/ovn-controller.c
ovn/controller/physical.c
ovn/controller/pipeline.c [deleted file]
ovn/controller/pipeline.h [deleted file]
ovn/lib/actions.c
ovn/northd/ovn-northd.c
ovn/ovn-architecture.7.xml
ovn/ovn-nb.xml
ovn/ovn-sb.ovsschema
ovn/ovn-sb.xml

index 07d66da..e46a7bc 100644 (file)
--- a/ovn/TODO
+++ b/ovn/TODO
@@ -48,7 +48,7 @@
     Currently, clients monitor the entire contents of a table.  It
     might make sense to allow clients to monitor only rows that
     satisfy specific criteria, e.g. to allow an ovn-controller to
-    receive only Pipeline rows for logical networks on its hypervisor.
+    receive only Logical_Flow rows for logical networks on its hypervisor.
 
 *** Reducing redundant data and code within ovsdb-server.
 
index 9ed6bec..9c91420 100644 (file)
@@ -6,12 +6,12 @@ ovn_controller_ovn_controller_SOURCES = \
        ovn/controller/chassis.h \
        ovn/controller/encaps.c \
        ovn/controller/encaps.h \
+       ovn/controller/lflow.c \
+       ovn/controller/lflow.h \
        ovn/controller/ofctrl.c \
        ovn/controller/ofctrl.h \
        ovn/controller/ovn-controller.c \
        ovn/controller/ovn-controller.h \
-       ovn/controller/pipeline.c \
-       ovn/controller/pipeline.h \
        ovn/controller/physical.c \
        ovn/controller/physical.h
 ovn_controller_ovn_controller_LDADD = ovn/lib/libovn.la lib/libopenvswitch.la
diff --git a/ovn/controller/lflow.c b/ovn/controller/lflow.c
new file mode 100644 (file)
index 0000000..bb2e7d1
--- /dev/null
@@ -0,0 +1,364 @@
+/* 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 "lflow.h"
+#include "dynamic-string.h"
+#include "ofctrl.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(lflow);
+\f
+/* Symbol table. */
+
+/* Contains "struct expr_symbol"s for fields supported by OVN lflows. */
+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_LOG_INPORT, NULL);
+    expr_symtab_add_string(&symtab, "outport", MFF_LOG_OUTPORT, NULL);
+
+    /* Registers.  We omit the registers that would otherwise overlap the
+     * reserved fields. */
+    for (enum mf_field_id id = MFF_REG0; id < MFF_REG0 + FLOW_N_REGS; id++) {
+        if (id != MFF_LOG_INPORT && id != MFF_LOG_OUTPORT) {
+            char name[8];
+
+            snprintf(name, sizeof name, "reg%d", id - MFF_REG0);
+            expr_symtab_add_field(&symtab, name, id, NULL, false);
+        }
+    }
+
+    /* 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);
+}
+\f
+/* 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
+ * Port_Binding 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;
+}
+
+/* Finds and returns the integer value corresponding to the given 'uuid', or 0
+ * if no such logical datapath exists. */
+uint32_t
+ldp_to_integer(const struct uuid *logical_datapath)
+{
+    const struct logical_datapath *ldp = ldp_lookup(logical_datapath);
+    return ldp ? ldp->integer : 0;
+}
+
+/* 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;
+
+    /* We don't handle the case where the logical datapaths wrap around. */
+    ovs_assert(next_integer);
+
+    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;
+}
+
+static void
+ldp_free(struct logical_datapath *ldp)
+{
+    simap_destroy(&ldp->ports);
+    hmap_remove(&logical_datapaths, &ldp->hmap_node);
+    free(ldp);
+}
+
+/* Iterates through all of the records in the Port_Binding 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_port_binding *binding;
+    SBREC_PORT_BINDING_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)) {
+            ldp_free(ldp);
+        }
+    }
+}
+
+static void
+ldp_destroy(void)
+{
+    struct logical_datapath *ldp, *next_ldp;
+    HMAP_FOR_EACH_SAFE (ldp, next_ldp, hmap_node, &logical_datapaths) {
+        ldp_free(ldp);
+    }
+}
+\f
+void
+lflow_init(void)
+{
+    symtab_init();
+}
+
+/* Translates logical flows in the Logical_Flow table in the OVN_SB database
+ * into OpenFlow flows, adding the OpenFlow flows to 'flow_table'.
+ *
+ * We put the logical flows into OpenFlow tables 16 through 47 (inclusive). */
+void
+lflow_run(struct controller_ctx *ctx, struct hmap *flow_table)
+{
+    struct hmap flows = HMAP_INITIALIZER(&flows);
+    uint32_t conj_id_ofs = 1;
+
+    ldp_run(ctx);
+
+    const struct sbrec_logical_flow *lflow;
+    SBREC_LOGICAL_FLOW_FOR_EACH (lflow, ctx->ovnsb_idl) {
+        /* Find the "struct logical_datapath" asssociated with this
+         * Logical_Flow 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(&lflow->logical_datapath);
+        if (!ldp) {
+            continue;
+        }
+
+        /* Translate OVN actions into OpenFlow actions. */
+        uint64_t ofpacts_stub[64 / 8];
+        struct ofpbuf ofpacts;
+        struct expr *prereqs;
+        uint8_t next_table_id;
+        char *error;
+
+        ofpbuf_use_stub(&ofpacts, ofpacts_stub, sizeof ofpacts_stub);
+        next_table_id = lflow->table_id < 31 ? lflow->table_id + 17 : 0;
+        error = actions_parse_string(lflow->actions, &symtab, &ldp->ports,
+                                     next_table_id, &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",
+                         lflow->actions, error);
+            free(error);
+            continue;
+        }
+
+        /* Translate OVN match into table of OpenFlow matches. */
+        struct hmap matches;
+        struct expr *expr;
+
+        expr = expr_parse_string(lflow->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",
+                         lflow->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) {
+                ofctrl_add_flow(flow_table, lflow->table_id + 16,
+                                lflow->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;
+                }
+                ofctrl_add_flow(flow_table, lflow->table_id + 16,
+                                lflow->priority, &m->match, &conj);
+                ofpbuf_uninit(&conj);
+            }
+        }
+
+        /* Clean up. */
+        expr_matches_destroy(&matches);
+        ofpbuf_uninit(&ofpacts);
+        conj_id_ofs += n_conjs;
+    }
+}
+
+void
+lflow_destroy(void)
+{
+    expr_symtab_destroy(&symtab);
+    ldp_destroy();
+}
diff --git a/ovn/controller/lflow.h b/ovn/controller/lflow.h
new file mode 100644 (file)
index 0000000..4659004
--- /dev/null
@@ -0,0 +1,51 @@
+/* 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_LFLOW_H
+#define OVN_LFLOW_H 1
+
+/* Logical_Flow table translation to OpenFlow
+ * ==========================================
+ *
+ * The Logical_Flow table obtained from the OVN_Southbound database works in
+ * terms of logical entities, that is, logical flows among logical datapaths
+ * and logical ports.  This code translates these logical flows into OpenFlow
+ * flows that, again, work in terms of logical entities implemented through
+ * OpenFlow extensions (e.g. registers represent the logical input and output
+ * ports).
+ *
+ * Physical-to-logical and logical-to-physical translation are implemented in
+ * physical.[ch] as separate OpenFlow tables that run before and after,
+ * respectively, the logical pipeline OpenFlow tables.
+ */
+
+#include <stdint.h>
+
+struct controller_ctx;
+struct hmap;
+struct uuid;
+
+/* Logical ports. */
+#define MFF_LOG_INPORT  MFF_REG6 /* Logical input port. */
+#define MFF_LOG_OUTPORT MFF_REG7 /* Logical output port. */
+
+void lflow_init(void);
+void lflow_run(struct controller_ctx *, struct hmap *flow_table);
+void lflow_destroy(void);
+
+uint32_t ldp_to_integer(const struct uuid *logical_datapath);
+
+#endif /* ovn/lflow.h */
index 5548e2a..84eeba8 100644 (file)
@@ -44,7 +44,7 @@
 #include "chassis.h"
 #include "encaps.h"
 #include "physical.h"
-#include "pipeline.h"
+#include "lflow.h"
 
 VLOG_DEFINE_THIS_MODULE(main);
 
@@ -238,7 +238,7 @@ main(int argc, char *argv[])
     sbrec_init();
 
     ofctrl_init();
-    pipeline_init();
+    lflow_init();
 
     /* Connect to OVS OVSDB instance.  We do not monitor all tables by
      * default, so modules must register their interest explicitly.  */
@@ -280,7 +280,7 @@ main(int argc, char *argv[])
 
         if (br_int) {
             struct hmap flow_table = HMAP_INITIALIZER(&flow_table);
-            pipeline_run(&ctx, &flow_table);
+            lflow_run(&ctx, &flow_table);
             if (chassis_id) {
                 physical_run(&ctx, br_int, chassis_id, &flow_table);
             }
@@ -332,7 +332,7 @@ main(int argc, char *argv[])
     }
 
     unixctl_server_destroy(unixctl);
-    pipeline_destroy();
+    lflow_destroy();
     ofctrl_destroy();
 
     idl_loop_destroy(&ovs_idl_loop);
index 77f5ff6..b0ed1c8 100644 (file)
 
 #include <config.h>
 #include "physical.h"
+#include "lflow.h"
 #include "match.h"
 #include "ofctrl.h"
 #include "ofp-actions.h"
 #include "ofpbuf.h"
 #include "ovn-controller.h"
 #include "ovn/lib/ovn-sb-idl.h"
-#include "pipeline.h"
 #include "simap.h"
 #include "vswitch-idl.h"
 
diff --git a/ovn/controller/pipeline.c b/ovn/controller/pipeline.c
deleted file mode 100644 (file)
index 867c323..0000000
+++ /dev/null
@@ -1,364 +0,0 @@
-/* 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 "ofctrl.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);
-\f
-/* 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_LOG_INPORT, NULL);
-    expr_symtab_add_string(&symtab, "outport", MFF_LOG_OUTPORT, NULL);
-
-    /* Registers.  We omit the registers that would otherwise overlap the
-     * reserved fields. */
-    for (enum mf_field_id id = MFF_REG0; id < MFF_REG0 + FLOW_N_REGS; id++) {
-        if (id != MFF_LOG_INPORT && id != MFF_LOG_OUTPORT) {
-            char name[8];
-
-            snprintf(name, sizeof name, "reg%d", id - MFF_REG0);
-            expr_symtab_add_field(&symtab, name, id, NULL, false);
-        }
-    }
-
-    /* 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);
-}
-\f
-/* 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
- * Port_Binding 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;
-}
-
-/* Finds and returns the integer value corresponding to the given 'uuid', or 0
- * if no such logical datapath exists. */
-uint32_t
-ldp_to_integer(const struct uuid *logical_datapath)
-{
-    const struct logical_datapath *ldp = ldp_lookup(logical_datapath);
-    return ldp ? ldp->integer : 0;
-}
-
-/* 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;
-
-    /* We don't handle the case where the logical datapaths wrap around. */
-    ovs_assert(next_integer);
-
-    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;
-}
-
-static void
-ldp_free(struct logical_datapath *ldp)
-{
-    simap_destroy(&ldp->ports);
-    hmap_remove(&logical_datapaths, &ldp->hmap_node);
-    free(ldp);
-}
-
-/* Iterates through all of the records in the Port_Binding 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_port_binding *binding;
-    SBREC_PORT_BINDING_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)) {
-            ldp_free(ldp);
-        }
-    }
-}
-
-static void
-ldp_destroy(void)
-{
-    struct logical_datapath *ldp, *next_ldp;
-    HMAP_FOR_EACH_SAFE (ldp, next_ldp, hmap_node, &logical_datapaths) {
-        ldp_free(ldp);
-    }
-}
-\f
-void
-pipeline_init(void)
-{
-    symtab_init();
-}
-
-/* Translates logical flows in the Pipeline table in the OVN_SB database
- * into OpenFlow flows, adding the OpenFlow flows to 'flow_table'.
- *
- * We put the Pipeline flows into OpenFlow tables 16 through 47 (inclusive). */
-void
-pipeline_run(struct controller_ctx *ctx, struct hmap *flow_table)
-{
-    struct hmap flows = HMAP_INITIALIZER(&flows);
-    uint32_t conj_id_ofs = 1;
-
-    ldp_run(ctx);
-
-    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) {
-            continue;
-        }
-
-        /* Translate OVN actions into OpenFlow actions. */
-        uint64_t ofpacts_stub[64 / 8];
-        struct ofpbuf ofpacts;
-        struct expr *prereqs;
-        uint8_t next_table_id;
-        char *error;
-
-        ofpbuf_use_stub(&ofpacts, ofpacts_stub, sizeof ofpacts_stub);
-        next_table_id = pipeline->table_id < 31 ? pipeline->table_id + 17 : 0;
-        error = actions_parse_string(pipeline->actions, &symtab, &ldp->ports,
-                                     next_table_id, &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) {
-                ofctrl_add_flow(flow_table, 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;
-                }
-                ofctrl_add_flow(flow_table, 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;
-    }
-}
-
-void
-pipeline_destroy(void)
-{
-    expr_symtab_destroy(&symtab);
-    ldp_destroy();
-}
diff --git a/ovn/controller/pipeline.h b/ovn/controller/pipeline.h
deleted file mode 100644 (file)
index 7d33341..0000000
+++ /dev/null
@@ -1,50 +0,0 @@
-/* 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
-
-/* Pipeline table translation to OpenFlow
- * ======================================
- *
- * The Pipeline table obtained from the OVN_Southbound database works in terms
- * of logical entities, that is, logical flows among logical datapaths and
- * logical ports.  This code translates these logical flows into OpenFlow flows
- * that, again, work in terms of logical entities implemented through OpenFlow
- * extensions (e.g. registers represent the logical input and output ports).
- *
- * Physical-to-logical and logical-to-physical translation are implemented in
- * physical.[ch] as separate OpenFlow tables that run before and after,
- * respectively, the logical pipeline OpenFlow tables.
- */
-
-#include <stdint.h>
-
-struct controller_ctx;
-struct hmap;
-struct uuid;
-
-/* Logical ports. */
-#define MFF_LOG_INPORT  MFF_REG6 /* Logical input port. */
-#define MFF_LOG_OUTPORT MFF_REG7 /* Logical output port. */
-
-void pipeline_init(void);
-void pipeline_run(struct controller_ctx *, struct hmap *flow_table);
-void pipeline_destroy(void);
-
-uint32_t ldp_to_integer(const struct uuid *logical_datapath);
-
-#endif /* ovn/pipeline.h */
index 28be688..ea8dfc9 100644 (file)
@@ -176,8 +176,8 @@ parse_actions(struct action_context *ctx)
 }
 
 /* Parses OVN actions, in the format described for the "actions" column in the
- * Pipeline table in ovn-sb(5), and appends the parsed versions of the actions
- * to 'ofpacts' as "struct ofpact"s.
+ * Logical_Flow table in ovn-sb(5), and appends the parsed versions of the
+ * actions to 'ofpacts' as "struct ofpact"s.
  *
  * 'symtab' provides a table of "struct expr_symbol"s to support (as one would
  * provide to expr_parse()).
index 757afca..aa58134 100644 (file)
@@ -120,35 +120,35 @@ macs_equal(char **binding_macs_, size_t b_n_macs,
     return (i == b_n_macs) ? true : false;
 }
 \f
-/* Pipeline generation.
+/* Logical flow generation.
  *
- * This code generates the Pipeline table in the southbound database, as a
+ * This code generates the Logical_Flow table in the southbound database, as a
  * function of most of the northbound database.
  */
 
-/* Enough context to add a Pipeline row, using pipeline_add(). */
-struct pipeline_ctx {
+/* Enough context to add a Logical_Flow row, using lflow_add(). */
+struct lflow_ctx {
     /* From northd_context. */
     struct ovsdb_idl *ovnsb_idl;
     struct ovsdb_idl_txn *ovnsb_txn;
 
-    /* Contains "struct pipeline_hash_node"s.  Used to figure out what existing
-     * Pipeline rows should be deleted: we index all of the Pipeline rows into
-     * this data structure, then as existing rows are generated we remove them.
-     * After generating all the rows, any remaining in 'pipeline_hmap' must be
-     * deleted from the database. */
-    struct hmap pipeline_hmap;
+    /* Contains "struct lflow_hash_node"s.  Used to figure out what existing
+     * Logical_Flow rows should be deleted: we index all of the Logical_Flow
+     * rows into this data structure, then as existing rows are generated we
+     * remove them.  After generating all the rows, any remaining in
+     * 'lflow_hmap' must be deleted from the database. */
+    struct hmap lflow_hmap;
 };
 
-/* A row in the Pipeline table, indexed by its full contents, */
-struct pipeline_hash_node {
+/* A row in the Logical_Flow table, indexed by its full contents, */
+struct lflow_hash_node {
     struct hmap_node node;
-    const struct sbrec_pipeline *pipeline;
+    const struct sbrec_logical_flow *lflow;
 };
 
 static size_t
-pipeline_hash(const struct uuid *logical_datapath, uint8_t table_id,
-              uint16_t priority, const char *match, const char *actions)
+lflow_hash(const struct uuid *logical_datapath, uint8_t table_id,
+          uint16_t priority, const char *match, const char *actions)
 {
     size_t hash = uuid_hash(logical_datapath);
     hash = hash_2words((table_id << 16) | priority, hash);
@@ -157,52 +157,52 @@ pipeline_hash(const struct uuid *logical_datapath, uint8_t table_id,
 }
 
 static size_t
-pipeline_hash_rec(const struct sbrec_pipeline *pipeline)
+lflow_hash_rec(const struct sbrec_logical_flow *lflow)
 {
-    return pipeline_hash(&pipeline->logical_datapath, pipeline->table_id,
-                         pipeline->priority, pipeline->match,
-                         pipeline->actions);
+    return lflow_hash(&lflow->logical_datapath, lflow->table_id,
+                      lflow->priority, lflow->match,
+                      lflow->actions);
 }
 
-/* Adds a row with the specified contents to the Pipeline table. */
+/* Adds a row with the specified contents to the Logical_Flow table. */
 static void
-pipeline_add(struct pipeline_ctx *ctx,
-             const struct nbrec_logical_switch *logical_datapath,
-             uint8_t table_id,
-             uint16_t priority,
-             const char *match,
-             const char *actions)
+lflow_add(struct lflow_ctx *ctx,
+          const struct nbrec_logical_switch *logical_datapath,
+          uint8_t table_id,
+          uint16_t priority,
+          const char *match,
+          const char *actions)
 {
-    struct pipeline_hash_node *hash_node;
+    struct lflow_hash_node *hash_node;
 
-    /* Check whether such a row already exists in the Pipeline table.  If so,
-     * remove it from 'ctx->pipeline_hmap' and we're done. */
+    /* Check whether such a row already exists in the Logical_Flow table.  If
+     * so, remove it from 'ctx->lflow_hmap' and we're done. */
     HMAP_FOR_EACH_WITH_HASH (hash_node, node,
-                             pipeline_hash(&logical_datapath->header_.uuid,
-                                           table_id, priority, match, actions),
-                             &ctx->pipeline_hmap) {
-        const struct sbrec_pipeline *pipeline = hash_node->pipeline;
-        if (uuid_equals(&pipeline->logical_datapath,
+                             lflow_hash(&logical_datapath->header_.uuid,
+                                        table_id, priority, match, actions),
+                             &ctx->lflow_hmap) {
+        const struct sbrec_logical_flow *lflow = hash_node->lflow;
+        if (uuid_equals(&lflow->logical_datapath,
                         &logical_datapath->header_.uuid)
-            && pipeline->table_id == table_id
-            && pipeline->priority == priority
-            && !strcmp(pipeline->match, match)
-            && !strcmp(pipeline->actions, actions)) {
-            hmap_remove(&ctx->pipeline_hmap, &hash_node->node);
+            && lflow->table_id == table_id
+            && lflow->priority == priority
+            && !strcmp(lflow->match, match)
+            && !strcmp(lflow->actions, actions)) {
+            hmap_remove(&ctx->lflow_hmap, &hash_node->node);
             free(hash_node);
             return;
         }
     }
 
-    /* No such Pipeline row.  Add one. */
-    const struct sbrec_pipeline *pipeline;
-    pipeline = sbrec_pipeline_insert(ctx->ovnsb_txn);
-    sbrec_pipeline_set_logical_datapath(pipeline,
-                                        logical_datapath->header_.uuid);
-    sbrec_pipeline_set_table_id(pipeline, table_id);
-    sbrec_pipeline_set_priority(pipeline, priority);
-    sbrec_pipeline_set_match(pipeline, match);
-    sbrec_pipeline_set_actions(pipeline, actions);
+    /* No such Logical_Flow row.  Add one. */
+    const struct sbrec_logical_flow *lflow;
+    lflow = sbrec_logical_flow_insert(ctx->ovnsb_txn);
+    sbrec_logical_flow_set_logical_datapath(lflow,
+                                            logical_datapath->header_.uuid);
+    sbrec_logical_flow_set_table_id(lflow, table_id);
+    sbrec_logical_flow_set_priority(lflow, priority);
+    sbrec_logical_flow_set_match(lflow, match);
+    sbrec_logical_flow_set_actions(lflow, actions);
 }
 
 /* Appends port security constraints on L2 address field 'eth_addr_field'
@@ -241,43 +241,43 @@ lport_is_enabled(const struct nbrec_logical_port *lport)
     return !lport->enabled || *lport->enabled;
 }
 
-/* Updates the Pipeline table in the OVN_SB database, constructing its contents
- * based on the OVN_NB database. */
+/* Updates the Logical_Flow table in the OVN_SB database, constructing its
+ * contents based on the OVN_NB database. */
 static void
-build_pipeline(struct northd_context *ctx)
+build_lflow(struct northd_context *ctx)
 {
-    struct pipeline_ctx pc = {
+    struct lflow_ctx pc = {
         .ovnsb_idl = ctx->ovnsb_idl,
         .ovnsb_txn = ctx->ovnsb_txn,
-        .pipeline_hmap = HMAP_INITIALIZER(&pc.pipeline_hmap)
+        .lflow_hmap = HMAP_INITIALIZER(&pc.lflow_hmap)
     };
 
-    /* Add all the Pipeline entries currently in the southbound database to
-     * 'pc.pipeline_hmap'.  We remove entries that we generate from the hmap,
+    /* Add all the Logical_Flow entries currently in the southbound database to
+     * 'pc.lflow_hmap'.  We remove entries that we generate from the hmap,
      * thus by the time we're done only entries that need to be removed
      * remain. */
-    const struct sbrec_pipeline *pipeline;
-    SBREC_PIPELINE_FOR_EACH (pipeline, ctx->ovnsb_idl) {
-        struct pipeline_hash_node *hash_node = xzalloc(sizeof *hash_node);
-        hash_node->pipeline = pipeline;
-        hmap_insert(&pc.pipeline_hmap, &hash_node->node,
-                    pipeline_hash_rec(pipeline));
+    const struct sbrec_logical_flow *lflow;
+    SBREC_LOGICAL_FLOW_FOR_EACH (lflow, ctx->ovnsb_idl) {
+        struct lflow_hash_node *hash_node = xzalloc(sizeof *hash_node);
+        hash_node->lflow = lflow;
+        hmap_insert(&pc.lflow_hmap, &hash_node->node,
+                    lflow_hash_rec(lflow));
     }
 
     /* Table 0: Admission control framework. */
     const struct nbrec_logical_switch *lswitch;
     NBREC_LOGICAL_SWITCH_FOR_EACH (lswitch, ctx->ovnnb_idl) {
         /* Logical VLANs not supported. */
-        pipeline_add(&pc, lswitch, 0, 100, "vlan.present", "drop;");
+        lflow_add(&pc, lswitch, 0, 100, "vlan.present", "drop;");
 
         /* Broadcast/multicast source address is invalid. */
-        pipeline_add(&pc, lswitch, 0, 100, "eth.src[40]", "drop;");
+        lflow_add(&pc, lswitch, 0, 100, "eth.src[40]", "drop;");
 
         /* Port security flows have priority 50 (see below) and will continue
          * to the next table if packet source is acceptable. */
 
         /* Otherwise drop the packet. */
-        pipeline_add(&pc, lswitch, 0, 0, "1", "drop;");
+        lflow_add(&pc, lswitch, 0, 0, "1", "drop;");
     }
 
     /* Table 0: Ingress port security. */
@@ -290,8 +290,8 @@ build_pipeline(struct northd_context *ctx)
             build_port_security("eth.src",
                                 lport->port_security, lport->n_port_security,
                                 &match);
-            pipeline_add(&pc, lswitch, 0, 50, ds_cstr(&match),
-                         lport_is_enabled(lport) ? "next;" : "drop;");
+            lflow_add(&pc, lswitch, 0, 50, ds_cstr(&match),
+                      lport_is_enabled(lport) ? "next;" : "drop;");
             ds_destroy(&match);
         }
     }
@@ -329,8 +329,8 @@ build_pipeline(struct northd_context *ctx)
                     ds_put_cstr(&unicast, "outport = ");
                     json_string_escape(lport->name, &unicast);
                     ds_put_cstr(&unicast, "; next;");
-                    pipeline_add(&pc, lswitch, 1, 50,
-                                 ds_cstr(&match), ds_cstr(&unicast));
+                    lflow_add(&pc, lswitch, 1, 50,
+                             ds_cstr(&match), ds_cstr(&unicast));
                     ds_destroy(&unicast);
                     ds_destroy(&match);
                 } else if (!strcmp(s, "unknown")) {
@@ -347,12 +347,12 @@ build_pipeline(struct northd_context *ctx)
         }
 
         ds_chomp(&bcast, ' ');
-        pipeline_add(&pc, lswitch, 1, 100, "eth.dst[40]", ds_cstr(&bcast));
+        lflow_add(&pc, lswitch, 1, 100, "eth.dst[40]", ds_cstr(&bcast));
         ds_destroy(&bcast);
 
         if (unknown.length) {
             ds_chomp(&unknown, ' ');
-            pipeline_add(&pc, lswitch, 1, 0, "1", ds_cstr(&unknown));
+            lflow_add(&pc, lswitch, 1, 0, "1", ds_cstr(&unknown));
         }
         ds_destroy(&unknown);
     }
@@ -363,19 +363,19 @@ build_pipeline(struct northd_context *ctx)
             const struct nbrec_acl *acl = lswitch->acls[i];
 
             NBREC_ACL_FOR_EACH (acl, ctx->ovnnb_idl) {
-                pipeline_add(&pc, lswitch, 2, acl->priority, acl->match,
-                             (!strcmp(acl->action, "allow") ||
-                              !strcmp(acl->action, "allow-related")
-                              ? "next;" : "drop;"));
+                lflow_add(&pc, lswitch, 2, acl->priority, acl->match,
+                          (!strcmp(acl->action, "allow") ||
+                           !strcmp(acl->action, "allow-related")
+                           ? "next;" : "drop;"));
             }
         }
 
-        pipeline_add(&pc, lswitch, 2, 0, "1", "next;");
+        lflow_add(&pc, lswitch, 2, 0, "1", "next;");
     }
 
     /* Table 3: Egress port security. */
     NBREC_LOGICAL_SWITCH_FOR_EACH (lswitch, ctx->ovnnb_idl) {
-        pipeline_add(&pc, lswitch, 3, 100, "eth.dst[40]", "output;");
+        lflow_add(&pc, lswitch, 3, 100, "eth.dst[40]", "output;");
 
         for (size_t i = 0; i < lswitch->n_ports; i++) {
             const struct nbrec_logical_port *lport = lswitch->ports[i];
@@ -388,21 +388,21 @@ build_pipeline(struct northd_context *ctx)
                                 lport->port_security, lport->n_port_security,
                                 &match);
 
-            pipeline_add(&pc, lswitch, 3, 50, ds_cstr(&match),
-                         lport_is_enabled(lport) ? "output;" : "drop;");
+            lflow_add(&pc, lswitch, 3, 50, ds_cstr(&match),
+                      lport_is_enabled(lport) ? "output;" : "drop;");
 
             ds_destroy(&match);
         }
     }
 
-    /* Delete any existing Pipeline rows that were not re-generated.  */
-    struct pipeline_hash_node *hash_node, *next_hash_node;
-    HMAP_FOR_EACH_SAFE (hash_node, next_hash_node, node, &pc.pipeline_hmap) {
-        hmap_remove(&pc.pipeline_hmap, &hash_node->node);
-        sbrec_pipeline_delete(hash_node->pipeline);
+    /* Delete any existing Logical_Flow rows that were not re-generated.  */
+    struct lflow_hash_node *hash_node, *next_hash_node;
+    HMAP_FOR_EACH_SAFE (hash_node, next_hash_node, node, &pc.lflow_hmap) {
+        hmap_remove(&pc.lflow_hmap, &hash_node->node);
+        sbrec_logical_flow_delete(hash_node->lflow);
         free(hash_node);
     }
-    hmap_destroy(&pc.pipeline_hmap);
+    hmap_destroy(&pc.lflow_hmap);
 }
 \f
 /*
@@ -628,7 +628,7 @@ ovnnb_db_changed(struct northd_context *ctx)
     VLOG_DBG("ovn-nb db contents have changed.");
 
     set_port_bindings(ctx);
-    build_pipeline(ctx);
+    build_lflow(ctx);
 }
 
 /*
@@ -824,16 +824,16 @@ main(int argc, char *argv[])
     ovsdb_idl_add_column(ovnsb_idl, &sbrec_port_binding_col_tunnel_key);
     ovsdb_idl_add_column(ovnsb_idl, &sbrec_port_binding_col_type);
     ovsdb_idl_add_column(ovnsb_idl, &sbrec_port_binding_col_options);
-    ovsdb_idl_add_column(ovnsb_idl, &sbrec_pipeline_col_logical_datapath);
-    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_pipeline_col_logical_datapath);
-    ovsdb_idl_add_column(ovnsb_idl, &sbrec_pipeline_col_table_id);
-    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_pipeline_col_table_id);
-    ovsdb_idl_add_column(ovnsb_idl, &sbrec_pipeline_col_priority);
-    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_pipeline_col_priority);
-    ovsdb_idl_add_column(ovnsb_idl, &sbrec_pipeline_col_match);
-    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_pipeline_col_match);
-    ovsdb_idl_add_column(ovnsb_idl, &sbrec_pipeline_col_actions);
-    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_pipeline_col_actions);
+    ovsdb_idl_add_column(ovnsb_idl, &sbrec_logical_flow_col_logical_datapath);
+    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_logical_flow_col_logical_datapath);
+    ovsdb_idl_add_column(ovnsb_idl, &sbrec_logical_flow_col_table_id);
+    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_logical_flow_col_table_id);
+    ovsdb_idl_add_column(ovnsb_idl, &sbrec_logical_flow_col_priority);
+    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_logical_flow_col_priority);
+    ovsdb_idl_add_column(ovnsb_idl, &sbrec_logical_flow_col_match);
+    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_logical_flow_col_match);
+    ovsdb_idl_add_column(ovnsb_idl, &sbrec_logical_flow_col_actions);
+    ovsdb_idl_omit_alert(ovnsb_idl, &sbrec_logical_flow_col_actions);
 
     /*
      * The loop here just runs the IDL in a loop waiting for the seqno to
index 13470bc..6e976ab 100644 (file)
       <code>ovn-northd</code> receives the OVN Northbound database update.
       In turn, it makes the corresponding updates to the OVN Southbound
       database, by adding rows to the OVN Southbound database
-      <code>Pipeline</code> table to reflect the new port, e.g. add a
+      <code>Logical_Flow</code> table to reflect the new port, e.g. add a
       flow to recognize that packets destined to the new port's MAC
       address should be delivered to it, and update the flow that
       delivers broadcast and multicast packets to include the new port.
 
     <li>
       On every hypervisor, <code>ovn-controller</code> receives the
-      <code>Pipeline</code> table updates that <code>ovn-northd</code> made
+      <code>Logical_Flow</code> table updates that <code>ovn-northd</code> made
       in the previous step.  As long as the VM that owns the VIF is powered off,
       <code>ovn-controller</code> cannot do much; it cannot, for example,
       arrange to send packets to or receive packets from the VIF, because the
       <code>Binding</code> table.  This provides <code>ovn-controller</code>
       the physical location of the logical port, so each instance updates the
       OpenFlow tables of its switch (based on logical datapath flows in the OVN
-      DB <code>Pipeline</code> table) so that packets to and from the VIF can
+      DB <code>Logical_Flow</code> table) so that packets to and from the VIF can
       be properly handled via tunnels.
     </li>
 
       <code>ovn-northd</code> receives the OVN Northbound update and in turn
       updates the OVN Southbound database accordingly, by removing or
       updating the rows from the OVN Southbound database
-      <code>Pipeline</code> table and <code>Binding</code> table that
+      <code>Logical_Flow</code> table and <code>Binding</code> table that
       were related to the now-destroyed VIF.
     </li>
 
     <li>
       On every hypervisor, <code>ovn-controller</code> receives the
-      <code>Pipeline</code> table updates that <code>ovn-northd</code> made
+      <code>Logical_Flow</code> table updates that <code>ovn-northd</code> made
       in the previous step.  <code>ovn-controller</code> updates OpenFlow tables
       to reflect the update, although there may not be much to do, since the VIF
       had already become unreachable when it was removed from the
       <code>ovn-northd</code> receives the OVN Northbound database update.
       In turn, it makes the corresponding updates to the OVN Southbound
       database, by adding rows to the OVN Southbound database's
-      <code>Pipeline</code> table to reflect the new port and also by
+      <code>Logical_Flow</code> table to reflect the new port and also by
       creating a new row in the <code>Binding</code> table and
       populating all its columns except the column that identifies the
       <code>chassis</code>.
       <code>ovn-northd</code> receives the OVN Northbound update and in turn
       updates the OVN Southbound database accordingly, by removing or
       updating the rows from the OVN Southbound database
-      <code>Pipeline</code> table that were related to the now-destroyed
+      <code>Logical_Flow</code> table that were related to the now-destroyed
       CIF.  It also deletes the row in the <code>Binding</code> table
       for that CIF.
     </li>
 
     <li>
       On every hypervisor, <code>ovn-controller</code> receives the
-      <code>Pipeline</code> table updates that <code>ovn-northd</code> made
-      in the previous step.  <code>ovn-controller</code> updates OpenFlow tables
-      to reflect the update.
+      <code>Logical_Flow</code> table updates that <code>ovn-northd</code> made
+      in the previous step.  <code>ovn-controller</code> updates OpenFlow
+      tables to reflect the update.
     </li>
   </ol>
 
index fac6ad0..7b9fbb2 100644 (file)
 
     <column name="match">
       The packets that the ACL should match, in the same expression
-      language used for the <ref column="match" table="Pipeline"
+      language used for the <ref column="match" table="Logical_Flow"
       db="OVN_Southbound"/> column in the OVN Southbound database's <ref
-      table="Pipeline" db="OVN_Southbound"/> table.  Match
+      table="Logical_Flow" db="OVN_Southbound"/> table.  Match
       <code>inport</code> and <code>outport</code> against names of
       logical ports within <ref column="lswitch"/> to implement ingress
       and egress ACLs, respectively.  In logical switches connected to
index 3c45658..e5c1812 100644 (file)
@@ -32,7 +32,7 @@
                                               "value": {"type": "string"},
                                               "min": 0,
                                               "max": "unlimited"}}}},
-        "Pipeline": {
+        "Logical_Flow": {
             "columns": {
                 "logical_datapath": {"type": "uuid"},
                 "table_id": {"type": {"key": {"type": "integer",
index 9dddf72..4860ec7 100644 (file)
@@ -74,7 +74,7 @@
   </p>
 
   <p>
-    The <ref table="Pipeline"/> table is currently the only LN table.
+    The <ref table="Logical_Flow"/> table is currently the only LN table.
   </p>
 
   <h3>Bindings data</h3>
     </column>
   </table>
 
-  <table name="Pipeline" title="Logical Network Pipeline">
+  <table name="Logical_Flow" title="Logical Network Flow">
     <p>
       Each row in this table represents one logical flow.  The cloud management
       system, via its OVN integration, populates this table with logical flows
     <column name="logical_datapath">
       The logical datapath to which the logical port belongs.  A logical
       datapath implements a logical pipeline via logical flows in the <ref
-      table="Pipeline"/> table.  (No table represents a logical datapath.)
+      table="Logical_Flow"/> table.  (No table represents a logical datapath.)
     </column>
 
     <column name="logical_port">