[ONOS-5565]Implementation of QosConfig and QueueConfig

Change-Id: I6a367b53cfca2e85e8aaa6cddb541d7b3ffccbc0
diff --git a/protocols/ovsdb/api/src/main/java/org/onosproject/ovsdb/controller/driver/DefaultOvsdbClient.java b/protocols/ovsdb/api/src/main/java/org/onosproject/ovsdb/controller/driver/DefaultOvsdbClient.java
index e8dbcb5..f60d48c 100644
--- a/protocols/ovsdb/api/src/main/java/org/onosproject/ovsdb/controller/driver/DefaultOvsdbClient.java
+++ b/protocols/ovsdb/api/src/main/java/org/onosproject/ovsdb/controller/driver/DefaultOvsdbClient.java
@@ -30,10 +30,13 @@
 
 import org.onlab.packet.IpAddress;
 import org.onosproject.net.DeviceId;
+import org.onosproject.net.PortNumber;
 import org.onosproject.net.behaviour.BridgeDescription;
 import org.onosproject.net.behaviour.ControllerInfo;
 import org.onosproject.net.behaviour.MirroringStatistics;
 import org.onosproject.net.behaviour.MirroringName;
+import org.onosproject.net.behaviour.QosId;
+import org.onosproject.net.behaviour.QueueId;
 import org.onosproject.ovsdb.controller.OvsdbBridge;
 import org.onosproject.ovsdb.controller.OvsdbClientService;
 import org.onosproject.ovsdb.controller.OvsdbInterface;
@@ -43,6 +46,8 @@
 import org.onosproject.ovsdb.controller.OvsdbPort;
 import org.onosproject.ovsdb.controller.OvsdbPortName;
 import org.onosproject.ovsdb.controller.OvsdbPortNumber;
+import org.onosproject.ovsdb.controller.OvsdbQos;
+import org.onosproject.ovsdb.controller.OvsdbQueue;
 import org.onosproject.ovsdb.controller.OvsdbRowStore;
 import org.onosproject.ovsdb.controller.OvsdbStore;
 import org.onosproject.ovsdb.controller.OvsdbTableStore;
@@ -50,6 +55,7 @@
 import org.onosproject.ovsdb.rfc.jsonrpc.Callback;
 import org.onosproject.ovsdb.rfc.message.OperationResult;
 import org.onosproject.ovsdb.rfc.message.TableUpdates;
+import org.onosproject.ovsdb.rfc.notation.Column;
 import org.onosproject.ovsdb.rfc.notation.Condition;
 import org.onosproject.ovsdb.rfc.notation.Mutation;
 import org.onosproject.ovsdb.rfc.notation.OvsdbMap;
@@ -70,6 +76,8 @@
 import org.onosproject.ovsdb.rfc.table.Mirror;
 import org.onosproject.ovsdb.rfc.table.OvsdbTable;
 import org.onosproject.ovsdb.rfc.table.Port;
+import org.onosproject.ovsdb.rfc.table.Qos;
+import org.onosproject.ovsdb.rfc.table.Queue;
 import org.onosproject.ovsdb.rfc.table.TableGenerator;
 import org.onosproject.ovsdb.rfc.utils.ConditionUtil;
 import org.onosproject.ovsdb.rfc.utils.FromJsonUtil;
@@ -81,6 +89,7 @@
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -449,7 +458,7 @@
         String portUuid = getPortUuid(portName, bridgeUuid);
         if (portUuid != null) {
             log.info("Port {} delete", portName);
-            deleteConfig(PORT, UUID, portUuid, BRIDGE, PORTS);
+            deleteConfig(PORT, UUID, portUuid, BRIDGE, PORTS, Uuid.uuid(portUuid));
         }
     }
 
@@ -575,7 +584,7 @@
         }
 
         removeControllers.forEach(c -> deleteConfig(CONTROLLER, UUID, c.getRow().uuid().value(),
-                                                    BRIDGE, "controller"));
+                                                    BRIDGE, "controller", c.getRow().uuid()));
         newControllers.stream().map(c -> {
             Controller controller = (Controller) TableGenerator
                     .createTable(dbSchema, OvsdbTable.CONTROLLER);
@@ -612,9 +621,261 @@
             log.warn("Could not find bridge in node", nodeId.getIpAddress());
             return;
         }
-        deleteConfig(BRIDGE, UUID, bridgeUuid, DATABASENAME, BRIDGES);
+        deleteConfig(BRIDGE, UUID, bridgeUuid, DATABASENAME, BRIDGES, Uuid.uuid(bridgeUuid));
     }
 
+    @Override
+    public void applyQos(PortNumber portNumber, String qosName) {
+        DatabaseSchema dbSchema = schema.get(DATABASENAME);
+        OvsdbRowStore portRowStore = getRowStore(DATABASENAME, PORT);
+        if (portRowStore == null) {
+            log.debug("The port uuid is null");
+            return;
+        }
+        OvsdbRowStore qosRowStore = getRowStore(DATABASENAME, QOS);
+        if (qosRowStore == null) {
+            log.debug("The qos uuid is null");
+            return;
+        }
+
+        // Due to Qos Table doesn't have a unique identifier except uuid, unlike
+        // Bridge or Port Table has a name column,in order to make the api more
+        // general, put qos name in external_ids column of Qos Table if this qos
+        // created by onos.
+        ConcurrentMap<String, Row> qosTableRows = qosRowStore.getRowStore();
+        ConcurrentMap<String, Row> portTableRows = portRowStore.getRowStore();
+        Row qosRow = qosTableRows.values().stream().filter(r -> {
+            OvsdbMap ovsdbMap = (OvsdbMap) (r.getColumn(EXTERNAL_ID).data());
+            return qosName.equals(ovsdbMap.map().get(QOS_EXTERNAL_ID_KEY));
+        }).findFirst().orElse(null);
+
+        Row portRow = portTableRows.values().stream()
+                .filter(r -> r.getColumn("name").data().equals(portNumber.name()))
+                .findFirst().orElse(null);
+        if (portRow != null && qosRow != null) {
+            String qosId = qosRow.uuid().value();
+            Uuid portUuid = portRow.uuid();
+            Map<String, Column> columns = new HashMap<>();
+            Row newPortRow = new Row(PORT, portUuid, columns);
+            Port newport = new Port(dbSchema, newPortRow);
+            columns.put(Port.PortColumn.QOS.columnName(), newport.getQosColumn());
+            newport.setQos(Uuid.uuid(qosId));
+            updateConfig(PORT, UUID, portUuid.value(), newport.getRow());
+        }
+    }
+
+    @Override
+    public void removeQos(PortNumber portNumber) {
+        DatabaseSchema dbSchema = schema.get(DATABASENAME);
+        OvsdbRowStore rowStore = getRowStore(DATABASENAME, PORT);
+        if (rowStore == null) {
+            log.debug("The qos uuid is null");
+            return;
+        }
+
+        ConcurrentMap<String, Row> ovsTableRows = rowStore.getRowStore();
+        Row portRow = ovsTableRows.values().stream()
+                .filter(r -> r.getColumn("name").data().equals(portNumber.name()))
+                .findFirst().orElse(null);
+        if (portRow == null) {
+            log.warn("Couldn't find port {} in ovsdb port table.", portNumber.name());
+            return;
+        }
+
+        OvsdbSet ovsdbSet = ((OvsdbSet) portRow.getColumn(PORT_QOS).data());
+        @SuppressWarnings("unchecked")
+        Set<Uuid> qosIdSet = ovsdbSet.set();
+        if (qosIdSet == null || qosIdSet.isEmpty()) {
+            return;
+        }
+        Uuid qosUuid = (Uuid) qosIdSet.toArray()[0];
+        Condition condition = ConditionUtil.isEqual(UUID, portRow.uuid());
+        List<Condition> conditions = Lists.newArrayList(condition);
+        Mutation mutation = MutationUtil.delete(PORT_QOS, qosUuid);
+        List<Mutation> mutations = Lists.newArrayList(mutation);
+
+        ArrayList<Operation> operations = Lists.newArrayList();
+        Mutate mutate = new Mutate(dbSchema.getTableSchema(PORT), conditions, mutations);
+        operations.add(mutate);
+        transactConfig(DATABASENAME, operations);
+    }
+
+    @Override
+    public boolean createQos(OvsdbQos ovsdbQos) {
+        DatabaseSchema dbSchema = schema.get(DATABASENAME);
+        Qos qos = (Qos) TableGenerator.createTable(dbSchema, OvsdbTable.QOS);
+        OvsdbRowStore rowStore = getRowStore(DATABASENAME, QOS);
+        if (rowStore == null) {
+            log.debug("The qos uuid is null");
+            return false;
+        }
+
+        ArrayList<Operation> operations = Lists.newArrayList();
+        Set<String> types = Sets.newHashSet();
+        Map<Long, Uuid> queues = Maps.newHashMap();
+
+        types.add(ovsdbQos.qosType());
+        qos.setOtherConfig(ovsdbQos.otherConfigs());
+        qos.setExternalIds(ovsdbQos.externalIds());
+        qos.setType(types);
+        if (ovsdbQos.qosQueues().isPresent()) {
+            for (Map.Entry<Long, String> entry : ovsdbQos.qosQueues().get().entrySet()) {
+                OvsdbRowStore queueRowStore = getRowStore(DATABASENAME, QUEUE);
+                if (queueRowStore != null) {
+                    ConcurrentMap<String, Row> queueTableRows = queueRowStore.getRowStore();
+                    Row queueRow = queueTableRows.values().stream().filter(r -> {
+                        OvsdbMap ovsdbMap = (OvsdbMap) (r.getColumn(EXTERNAL_ID).data());
+                        return entry.getValue().equals(ovsdbMap.map().get(QUEUE_EXTERNAL_ID_KEY));
+                    }).findFirst().orElse(null);
+                    if (queueRow != null) {
+                        queues.put(entry.getKey(), queueRow.uuid());
+                    }
+                }
+            }
+            qos.setQueues(queues);
+        }
+
+        Insert qosInsert = new Insert(dbSchema.getTableSchema(QOS), QOS, qos.getRow());
+        operations.add(qosInsert);
+        try {
+            transactConfig(DATABASENAME, operations).get();
+        } catch (InterruptedException | ExecutionException e) {
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public void dropQos(QosId qosId) {
+        OvsdbRowStore rowStore = getRowStore(DATABASENAME, QOS);
+        if (rowStore != null) {
+            ConcurrentMap<String, Row> qosTableRows = rowStore.getRowStore();
+            Row qosRow = qosTableRows.values().stream().filter(r -> {
+                        OvsdbMap ovsdbMap = (OvsdbMap) (r.getColumn(EXTERNAL_ID).data());
+                        return qosId.name().equals(ovsdbMap.map().get(QOS_EXTERNAL_ID_KEY));
+                    }).findFirst().orElse(null);
+            if (qosRow != null) {
+                deleteConfig(QOS, UUID, qosRow.uuid().value(), PORT, PORT_QOS, qosRow.uuid());
+            }
+        }
+    }
+    @Override
+    public OvsdbQos getQos(QosId qosId) {
+        Set<OvsdbQos> ovsdbQoses = getQoses();
+        return ovsdbQoses.stream().filter(r ->
+                qosId.name().equals(r.externalIds().get(QOS_EXTERNAL_ID_KEY))).
+                findFirst().orElse(null);
+    }
+
+    @Override
+    public Set<OvsdbQos> getQoses() {
+        Set<OvsdbQos> ovsdbQoses = new HashSet<>();
+        OvsdbRowStore rowStore = getRowStore(DATABASENAME, QOS);
+        if (rowStore == null) {
+            log.debug("The qos uuid is null");
+            return ovsdbQoses;
+        }
+        ConcurrentMap<String, Row> rows = rowStore.getRowStore();
+        for (String uuid : rows.keySet()) {
+            Row row = getRow(DATABASENAME, QOS, uuid);
+            OvsdbQos ovsdbQos = getOvsdbQos(row);
+            if (ovsdbQos != null) {
+                ovsdbQoses.add(ovsdbQos);
+            }
+        }
+        return ovsdbQoses;
+    }
+
+    @Override
+    public boolean createQueue(OvsdbQueue ovsdbQueue) {
+        DatabaseSchema dbSchema = schema.get(DATABASENAME);
+        Queue queue = (Queue) TableGenerator.createTable(dbSchema, OvsdbTable.QUEUE);
+        ArrayList<Operation> operations = Lists.newArrayList();
+        OvsdbRowStore rowStore = getRowStore(DATABASENAME, QUEUE);
+        if (rowStore == null) {
+            log.debug("The queue uuid is null");
+            return false;
+        }
+
+        if (ovsdbQueue.dscp().isPresent()) {
+            queue.setDscp(ImmutableSet.of(ovsdbQueue.dscp().get()));
+        }
+        queue.setOtherConfig(ovsdbQueue.otherConfigs());
+        queue.setExternalIds(ovsdbQueue.externalIds());
+        Insert queueInsert = new Insert(dbSchema.getTableSchema(QUEUE), QUEUE, queue.getRow());
+        operations.add(queueInsert);
+
+        try {
+            transactConfig(DATABASENAME, operations).get();
+        } catch (InterruptedException | ExecutionException e) {
+            log.error("createQueue transactConfig get exception !");
+        }
+        return true;
+    }
+
+    @Override
+    public void dropQueue(QueueId queueId) {
+        OvsdbRowStore queueRowStore = getRowStore(DATABASENAME, QUEUE);
+        if (queueRowStore == null) {
+            return;
+        }
+
+        ConcurrentMap<String, Row> queueTableRows = queueRowStore.getRowStore();
+        Row queueRow = queueTableRows.values().stream().filter(r -> {
+            OvsdbMap ovsdbMap = (OvsdbMap) (r.getColumn(EXTERNAL_ID).data());
+            return queueId.name().equals(ovsdbMap.map().get(QUEUE_EXTERNAL_ID_KEY));
+        }).findFirst().orElse(null);
+        if (queueRow == null) {
+            return;
+        }
+
+        String queueUuid = queueRow.uuid().value();
+        OvsdbRowStore qosRowStore = getRowStore(DATABASENAME, QOS);
+        if (qosRowStore != null) {
+            Map<Long, Uuid> queueMap = new HashMap<>();
+            ConcurrentMap<String, Row> qosTableRows = qosRowStore.getRowStore();
+            qosTableRows.values().stream().filter(r -> {
+                Map<Integer, Uuid> ovsdbMap = ((OvsdbMap) r.getColumn(QUEUES).data()).map();
+                Set<Integer> keySet = ovsdbMap.keySet();
+                for (Integer keyId : keySet) {
+                    if (ovsdbMap.get(keyId).equals(Uuid.uuid(queueUuid))) {
+                        queueMap.put(keyId.longValue(), Uuid.uuid(queueUuid));
+                        return true;
+                    }
+                }
+                return false;
+            }).findFirst().orElse(null);
+            deleteConfig(QUEUE, UUID, queueUuid, QOS, QUEUES, OvsdbMap.ovsdbMap(queueMap));
+        } else {
+            deleteConfig(QUEUE, UUID, queueUuid, null, null, null);
+        }
+    }
+    @Override
+    public OvsdbQueue getQueue(QueueId queueId) {
+        Set<OvsdbQueue> ovsdbQueues = getQueues();
+        return ovsdbQueues.stream().filter(r ->
+                queueId.name().equals(r.externalIds().get(QUEUE_EXTERNAL_ID_KEY))).
+                findFirst().orElse(null);
+    }
+
+    @Override
+    public Set<OvsdbQueue> getQueues() {
+        Set<OvsdbQueue> ovsdbqueues = new HashSet<>();
+        OvsdbRowStore rowStore = getRowStore(DATABASENAME, QUEUE);
+        if (rowStore == null) {
+            log.debug("The queue uuid is null");
+            return ovsdbqueues;
+        }
+        ConcurrentMap<String, Row> rows = rowStore.getRowStore();
+        for (String uuid : rows.keySet()) {
+            Row row = getRow(DATABASENAME, QUEUE, uuid);
+            OvsdbQueue ovsdbQueue = getOvsdbQueue(row);
+            if (ovsdbQueue != null) {
+                ovsdbqueues.add(ovsdbQueue);
+            }
+        }
+        return ovsdbqueues;
+    }
     /**
      * Creates a mirror port. Mirrors the traffic
      * that goes to selectDstPort or comes from
@@ -745,7 +1006,7 @@
         String mirrorUuid = getMirrorUuid(mirroringName.name());
         if (mirrorUuid != null) {
             log.info("Deleted mirror {}", mirroringName.name());
-            deleteConfig(MIRROR, UUID, mirrorUuid, BRIDGE, MIRRORS);
+            deleteConfig(MIRROR, UUID, mirrorUuid, BRIDGE, MIRRORS, Uuid.uuid(mirrorUuid));
         }
         log.warn("Unable to delete {}", mirroringName.name());
         return;
@@ -834,7 +1095,7 @@
 
         if (bridgeId.isPresent()) {
             String portId = getPortUuid(ifaceName, bridgeId.get());
-            deleteConfig(PORT, UUID, portId, BRIDGE, PORTS);
+            deleteConfig(PORT, UUID, portId, BRIDGE, PORTS, Uuid.uuid(portId));
             return true;
         } else {
             log.warn("Unable to find the interface with name {}", ifaceName);
@@ -850,26 +1111,25 @@
      * @param childUuid        child row uuid
      * @param parentTableName  parent table name
      * @param parentColumnName parent column
+     * @param referencedValue  referenced value
      */
     private void deleteConfig(String childTableName, String childColumnName,
                               String childUuid, String parentTableName,
-                              String parentColumnName) {
+                              String parentColumnName, Object referencedValue) {
         DatabaseSchema dbSchema = schema.get(DATABASENAME);
         TableSchema childTableSchema = dbSchema.getTableSchema(childTableName);
 
         ArrayList<Operation> operations = Lists.newArrayList();
-        if (parentTableName != null && parentColumnName != null) {
+        if (parentTableName != null && parentColumnName != null && referencedValue != null) {
             TableSchema parentTableSchema = dbSchema
                     .getTableSchema(parentTableName);
             ColumnSchema parentColumnSchema = parentTableSchema
                     .getColumnSchema(parentColumnName);
             List<Mutation> mutations = Lists.newArrayList();
-            Mutation mutation = MutationUtil.delete(parentColumnSchema.name(),
-                                                    Uuid.uuid(childUuid));
+            Mutation mutation = MutationUtil.delete(parentColumnSchema.name(), referencedValue);
             mutations.add(mutation);
             List<Condition> conditions = Lists.newArrayList();
-            Condition condition = ConditionUtil.includes(parentColumnName,
-                                                         Uuid.uuid(childUuid));
+            Condition condition = ConditionUtil.includes(parentColumnName, referencedValue);
             conditions.add(condition);
             Mutate op = new Mutate(parentTableSchema, conditions, mutations);
             operations.add(op);
@@ -1134,7 +1394,6 @@
         SettableFuture<List<JsonNode>> sf = SettableFuture.create();
         requestResult.put(id, sf);
         requestMethod.put(id, "transact");
-
         channel.writeAndFlush(transactString);
         return sf;
     }
@@ -1351,6 +1610,50 @@
         return OvsdbBridge.builder().name(bridgeName).datapathId(datapathId).build();
     }
 
+    private OvsdbQos getOvsdbQos(Row row) {
+        DatabaseSchema dbSchema = getDatabaseSchema(DATABASENAME);
+        Qos qos = (Qos) TableGenerator.getTable(dbSchema, row, OvsdbTable.QOS);
+        if (qos == null) {
+            return null;
+        }
+
+        String type = (String) qos.getTypeColumn().data();
+        Map<String, String> otherConfigs;
+        Map<String, String> externalIds;
+        Map<Long, String> queues;
+
+        otherConfigs = ((OvsdbMap) qos.getOtherConfigColumn().data()).map();
+        externalIds  = ((OvsdbMap) qos.getExternalIdsColumn().data()).map();
+        queues = ((OvsdbMap) qos.getQueuesColumn().data()).map();
+        return OvsdbQos.builder().qosType(type).
+                queues(queues).otherConfigs(otherConfigs).
+                externalIds(externalIds).build();
+    }
+
+    private OvsdbQueue getOvsdbQueue(Row row) {
+        DatabaseSchema dbSchema = getDatabaseSchema(DATABASENAME);
+        Queue queue = (Queue) TableGenerator.getTable(dbSchema, row, OvsdbTable.QUEUE);
+        if (queue == null) {
+            return null;
+        }
+
+        OvsdbSet dscpOvsdbSet = ((OvsdbSet) queue.getDscpColumn().data());
+        @SuppressWarnings("unchecked")
+        Set<String> dscpSet = dscpOvsdbSet.set();
+        Long dscp = null;
+        if (dscpSet != null && !dscpSet.isEmpty()) {
+            dscp = Long.valueOf((String) dscpSet.toArray()[0]);
+        }
+
+        Map<String, String> otherConfigs;
+        Map<String, String> externalIds;
+
+        otherConfigs = ((OvsdbMap) queue.getOtherConfigColumn().data()).map();
+        externalIds  = ((OvsdbMap) queue.getExternalIdsColumn().data()).map();
+        return OvsdbQueue.builder().dscp(dscp).
+                otherConfigs(otherConfigs).externalIds(externalIds).build();
+    }
+
     private long getOfPort(Interface intf) {
         OvsdbSet ofPortSet = (OvsdbSet) intf.getOpenFlowPortColumn().data();
         @SuppressWarnings("unchecked")