ONOS-7077 Openflow 1.5 OXS and stat trigger support

Change-Id: I006bcd3d8eac451a780c7e5c69a12298ead14281
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilder.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilder.java
index 4ec4a69..a281927 100644
--- a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilder.java
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilder.java
@@ -134,8 +134,9 @@
             return new FlowModBuilderVer10(flowRule, factory, xid, driverService);
         case OF_13:
         case OF_14:
-        case OF_15:
             return new FlowModBuilderVer13(flowRule, factory, xid, driverService);
+        case OF_15:
+            return new FlowModBuilderVer15(flowRule, factory, xid, driverService);
         default:
             throw new UnsupportedOperationException(
                     "No flow mod builder for protocol version " + factory.getVersion());
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilderVer13.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilderVer13.java
index 5381af7..ad601c9 100644
--- a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilderVer13.java
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilderVer13.java
@@ -100,10 +100,10 @@
  */
 public class FlowModBuilderVer13 extends FlowModBuilder {
 
-    private final Logger log = LoggerFactory.getLogger(getClass());
-    private static final int OFPCML_NO_BUFFER = 0xffff;
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+    protected static final int OFPCML_NO_BUFFER = 0xffff;
 
-    private final TrafficTreatment treatment;
+    protected final TrafficTreatment treatment;
 
     /**
      * Constructor for a flow mod builder for OpenFlow 1.3.
@@ -295,24 +295,24 @@
         return actions;
     }
 
-    private OFInstruction buildTableGoto(Instructions.TableTypeTransition i) {
+    protected OFInstruction buildTableGoto(Instructions.TableTypeTransition i) {
         OFInstruction instruction = factory().instructions().gotoTable(
                 TableId.of(i.tableId()));
         return instruction;
     }
 
-    private OFInstruction buildMetadata(Instructions.MetadataInstruction m) {
+    protected OFInstruction buildMetadata(Instructions.MetadataInstruction m) {
         OFInstruction instruction = factory().instructions().writeMetadata(
                 U64.of(m.metadata()), U64.of(m.metadataMask()));
         return instruction;
     }
 
-    private OFInstruction buildMeter(Instructions.MeterInstruction metered) {
+    protected OFInstruction buildMeter(Instructions.MeterInstruction metered) {
         return factory().instructions().meter(metered.meterId().id());
     }
 
 
-    private OFAction buildL0Modification(Instruction i) {
+    protected OFAction buildL0Modification(Instruction i) {
         L0ModificationInstruction l0m = (L0ModificationInstruction) i;
         OFOxm<?> oxm = null;
         switch (l0m.subtype()) {
@@ -340,7 +340,7 @@
         return null;
     }
 
-    private OFAction buildL1Modification(Instruction i) {
+    protected OFAction buildL1Modification(Instruction i) {
         L1ModificationInstruction l1m = (L1ModificationInstruction) i;
         OFOxm<?> oxm = null;
         switch (l1m.subtype()) {
@@ -365,7 +365,7 @@
         return null;
     }
 
-    private OFAction buildL2Modification(Instruction i) {
+    protected OFAction buildL2Modification(Instruction i) {
         L2ModificationInstruction l2m = (L2ModificationInstruction) i;
         ModEtherInstruction eth;
         OFOxm<?> oxm = null;
@@ -430,7 +430,7 @@
         return null;
     }
 
-    private OFAction buildL3Modification(Instruction i) {
+    protected OFAction buildL3Modification(Instruction i) {
         L3ModificationInstruction l3m = (L3ModificationInstruction) i;
         ModIPInstruction ip;
         Ip4Address ip4;
@@ -493,7 +493,7 @@
         return null;
     }
 
-    private OFAction buildL4Modification(Instruction i) {
+    protected OFAction buildL4Modification(Instruction i) {
         L4ModificationInstruction l4m = (L4ModificationInstruction) i;
         ModTransportPortInstruction tp;
         OFOxm<?> oxm = null;
@@ -525,7 +525,7 @@
         return null;
     }
 
-    private OFAction buildExtensionAction(ExtensionTreatment i) {
+    protected OFAction buildExtensionAction(ExtensionTreatment i) {
         if (!driverService.isPresent()) {
             log.error("No driver service present");
             return null;
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilderVer15.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilderVer15.java
new file mode 100644
index 0000000..7ba893a
--- /dev/null
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/FlowModBuilderVer15.java
@@ -0,0 +1,301 @@
+/*
+ * Copyright 2017-present Open Networking Foundation
+ *
+ * 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.
+ */
+package org.onosproject.provider.of.flow.impl;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.onosproject.net.PortNumber;
+import org.onosproject.net.driver.DriverService;
+import org.onosproject.net.flow.FlowRule;
+import org.onosproject.net.flow.StatTriggerField;
+import org.onosproject.net.flow.StatTriggerFlag;
+import org.onosproject.net.flow.instructions.Instruction;
+import org.onosproject.net.flow.instructions.Instructions;
+import org.projectfloodlight.openflow.protocol.OFFactory;
+import org.projectfloodlight.openflow.protocol.OFFlowAdd;
+import org.projectfloodlight.openflow.protocol.OFFlowMod;
+import org.projectfloodlight.openflow.protocol.OFFlowModFlags;
+import org.projectfloodlight.openflow.protocol.OFOxsList;
+import org.projectfloodlight.openflow.protocol.OFStatTriggerFlags;
+import org.projectfloodlight.openflow.protocol.action.OFAction;
+import org.projectfloodlight.openflow.protocol.action.OFActionGroup;
+import org.projectfloodlight.openflow.protocol.action.OFActionMeter;
+import org.projectfloodlight.openflow.protocol.action.OFActionOutput;
+import org.projectfloodlight.openflow.protocol.action.OFActionSetQueue;
+import org.projectfloodlight.openflow.protocol.instruction.OFInstruction;
+import org.projectfloodlight.openflow.protocol.match.Match;
+import org.projectfloodlight.openflow.protocol.oxs.OFOxs;
+import org.projectfloodlight.openflow.types.OFBufferId;
+import org.projectfloodlight.openflow.types.OFGroup;
+import org.projectfloodlight.openflow.types.OFPort;
+import org.projectfloodlight.openflow.types.TableId;
+import org.projectfloodlight.openflow.types.U32;
+import org.projectfloodlight.openflow.types.U64;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.projectfloodlight.openflow.protocol.OFStatTriggerFlags.ONLY_FIRST;
+import static org.projectfloodlight.openflow.protocol.OFStatTriggerFlags.PERIODIC;
+
+/**
+ * Flow mod builder for OpenFlow 1.5+.
+ */
+public class FlowModBuilderVer15 extends FlowModBuilderVer13 {
+
+    /**
+     * Constructor for a flow mod builder for OpenFlow 1.5.
+     *
+     * @param flowRule      the flow rule to transform into a flow mod
+     * @param factory       the OpenFlow factory to use to build the flow mod
+     * @param xid           the transaction ID
+     * @param driverService the device driver service
+     */
+    protected FlowModBuilderVer15(FlowRule flowRule, OFFactory factory,
+                                  Optional<Long> xid,
+                                  Optional<DriverService> driverService) {
+        super(flowRule, factory, xid, driverService);
+    }
+
+    @Override
+    public OFFlowMod buildFlowAdd() {
+        Match match = buildMatch();
+        List<OFAction> deferredActions = buildActions(treatment.deferred(), false);
+        List<OFAction> immediateActions = buildActions(treatment.immediate(), true);
+        List<OFInstruction> instructions = Lists.newLinkedList();
+
+        if (treatment.clearedDeferred()) {
+            instructions.add(factory().instructions().clearActions());
+        }
+        if (!immediateActions.isEmpty()) {
+            instructions.add(factory().instructions().applyActions(immediateActions));
+        }
+        if (!deferredActions.isEmpty()) {
+            instructions.add(factory().instructions().writeActions(deferredActions));
+        }
+        if (treatment.tableTransition() != null) {
+            instructions.add(buildTableGoto(treatment.tableTransition()));
+        }
+        if (treatment.writeMetadata() != null) {
+            instructions.add(buildMetadata(treatment.writeMetadata()));
+        }
+        if (treatment.statTrigger() != null) {
+            instructions.add(buildStatTrigger(treatment.statTrigger()));
+        }
+
+
+
+        long cookie = flowRule().id().value();
+
+        OFFlowAdd fm = factory().buildFlowAdd()
+                .setXid(xid)
+                .setCookie(U64.of(cookie))
+                .setBufferId(OFBufferId.NO_BUFFER)
+                .setInstructions(instructions)
+                .setMatch(match)
+                .setFlags(Collections.singleton(OFFlowModFlags.SEND_FLOW_REM))
+                .setPriority(flowRule().priority())
+                .setTableId(TableId.of(flowRule().tableId()))
+                .setHardTimeout(flowRule().hardTimeout())
+                .build();
+
+        return fm;
+    }
+
+    @Override
+    public OFFlowMod buildFlowMod() {
+        Match match = buildMatch();
+        List<OFAction> deferredActions = buildActions(treatment.deferred(), false);
+        List<OFAction> immediateActions = buildActions(treatment.immediate(), true);
+        List<OFInstruction> instructions = Lists.newLinkedList();
+
+
+        if (!immediateActions.isEmpty()) {
+            instructions.add(factory().instructions().applyActions(immediateActions));
+        }
+        if (treatment.clearedDeferred()) {
+            instructions.add(factory().instructions().clearActions());
+        }
+        if (!deferredActions.isEmpty()) {
+            instructions.add(factory().instructions().writeActions(deferredActions));
+        }
+        if (treatment.tableTransition() != null) {
+            instructions.add(buildTableGoto(treatment.tableTransition()));
+        }
+        if (treatment.writeMetadata() != null) {
+            instructions.add(buildMetadata(treatment.writeMetadata()));
+        }
+        if (treatment.statTrigger() != null) {
+            instructions.add(buildStatTrigger(treatment.statTrigger()));
+        }
+
+        long cookie = flowRule().id().value();
+
+        OFFlowMod fm = factory().buildFlowModify()
+                .setXid(xid)
+                .setCookie(U64.of(cookie))
+                .setBufferId(OFBufferId.NO_BUFFER)
+                .setInstructions(instructions)
+                .setMatch(match)
+                .setFlags(Collections.singleton(OFFlowModFlags.SEND_FLOW_REM))
+                .setPriority(flowRule().priority())
+                .setTableId(TableId.of(flowRule().tableId()))
+                .setHardTimeout(flowRule().hardTimeout())
+                .build();
+
+        return fm;
+    }
+
+    private List<OFAction> buildActions(List<Instruction> treatments, Boolean immediateActions) {
+        if (treatment == null) {
+            return Collections.emptyList();
+        }
+
+        boolean tableFound = false;
+        List<OFAction> actions = new LinkedList<>();
+
+        //Meter action handling
+        if (null != treatment.metered() && immediateActions) {
+            OFAction meterAction = buildMultipleMeterAction(treatment.metered());
+            actions.add(meterAction);
+        }
+
+        for (Instruction i : treatments) {
+            switch (i.type()) {
+                case NOACTION:
+                    return Collections.emptyList();
+                case L0MODIFICATION:
+                    actions.add(buildL0Modification(i));
+                    break;
+                case L1MODIFICATION:
+                    actions.add(buildL1Modification(i));
+                    break;
+                case L2MODIFICATION:
+                    actions.add(buildL2Modification(i));
+                    break;
+                case L3MODIFICATION:
+                    actions.add(buildL3Modification(i));
+                    break;
+                case L4MODIFICATION:
+                    actions.add(buildL4Modification(i));
+                    break;
+                case OUTPUT:
+                    Instructions.OutputInstruction out = (Instructions.OutputInstruction) i;
+                    OFActionOutput.Builder action = factory().actions().buildOutput()
+                            .setPort(OFPort.of((int) out.port().toLong()));
+                    if (out.port().equals(PortNumber.CONTROLLER)) {
+                        action.setMaxLen(OFPCML_NO_BUFFER);
+                    }
+                    actions.add(action.build());
+                    break;
+                case GROUP:
+                    Instructions.GroupInstruction group = (Instructions.GroupInstruction) i;
+                    OFActionGroup.Builder groupBuilder = factory().actions().buildGroup()
+                            .setGroup(OFGroup.of(group.groupId().id()));
+                    actions.add(groupBuilder.build());
+                    break;
+                case QUEUE:
+                    Instructions.SetQueueInstruction queue = (Instructions.SetQueueInstruction) i;
+                    OFActionSetQueue.Builder queueBuilder = factory().actions().buildSetQueue()
+                            .setQueueId(queue.queueId());
+                    actions.add(queueBuilder.build());
+                    break;
+                case TABLE:
+                    //FIXME: should not occur here.
+                    tableFound = true;
+                    break;
+                case EXTENSION:
+                    actions.add(buildExtensionAction(((Instructions.ExtensionInstructionWrapper) i)
+                            .extensionInstruction()));
+                    break;
+                default:
+                    log.warn("Instruction type {} not yet implemented.", i.type());
+            }
+        }
+
+        if (tableFound && actions.isEmpty()) {
+            // handles the case where there are no actions, but there is
+            // a goto instruction for the next table
+            return Collections.emptyList();
+        }
+        return actions;
+    }
+
+    private OFOxsList getOFOxsList(Map<StatTriggerField, Long> statTriggerMap) {
+        OFFactory factory = factory();
+        List<OFOxs<?>> ofOxsList = Lists.newArrayList();
+        for (Map.Entry<StatTriggerField, Long> entry : statTriggerMap.entrySet()) {
+            switch (entry.getKey()) {
+                case DURATION:
+                    ofOxsList.add(factory.oxss().buildDuration().setValue(U64.of(entry.getValue())).build());
+                    break;
+                case IDLE_TIME:
+                    ofOxsList.add(factory.oxss().buildIdleTime().setValue(U64.of(entry.getValue())).build());
+                    break;
+                case BYTE_COUNT:
+                    ofOxsList.add(factory.oxss().buildByteCount().setValue(U64.of(entry.getValue())).build());
+                    break;
+                case FLOW_COUNT:
+                    ofOxsList.add(factory.oxss().buildFlowCount().setValue(U32.of(entry.getValue())).build());
+                    break;
+                case PACKET_COUNT:
+                    ofOxsList.add(factory.oxss().buildPacketCount().setValue(U64.of(entry.getValue())).build());
+                    break;
+                default:
+                    log.warn("Unsupported Stat Trigger field");
+                    break;
+            }
+        }
+        return OFOxsList.ofList(ofOxsList);
+    }
+
+    private Set<OFStatTriggerFlags> getStatTriggerFlag(StatTriggerFlag flag) {
+        Set<OFStatTriggerFlags> statTriggerFlagsSet = Sets.newHashSet();
+        switch (flag) {
+            case PERIODIC:
+                statTriggerFlagsSet.add(PERIODIC);
+                break;
+            case ONLY_FIRST:
+                statTriggerFlagsSet.add(ONLY_FIRST);
+                break;
+            default:
+                break;
+        }
+        return statTriggerFlagsSet;
+    }
+
+    /**
+     * Meter action builder.
+     *
+     * @param meterInstruction meter instruction
+     * @return meter action
+     */
+    protected OFAction buildMultipleMeterAction(Instructions.MeterInstruction meterInstruction) {
+        OFActionMeter.Builder meterBuilder = factory().actions().buildMeter()
+                .setMeterId(meterInstruction.meterId().id());
+        return meterBuilder.build();
+    }
+
+    protected OFInstruction buildStatTrigger(Instructions.StatTriggerInstruction s) {
+        OFInstruction instruction = factory().instructions().statTrigger(getStatTriggerFlag(s.getStatTriggerFlag()),
+                getOFOxsList(s.getStatTriggerFieldMap()));
+        return instruction;
+    }
+}
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java
index 9dc03cd..7f472ad 100644
--- a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java
@@ -63,6 +63,7 @@
 import org.projectfloodlight.openflow.protocol.OFBadRequestCode;
 import org.projectfloodlight.openflow.protocol.OFBarrierRequest;
 import org.projectfloodlight.openflow.protocol.OFErrorMsg;
+import org.projectfloodlight.openflow.protocol.OFFlowLightweightStatsReply;
 import org.projectfloodlight.openflow.protocol.OFFlowMod;
 import org.projectfloodlight.openflow.protocol.OFFlowRemoved;
 import org.projectfloodlight.openflow.protocol.OFFlowStatsReply;
@@ -142,6 +143,7 @@
 
     private final Timer timer = new Timer("onos-openflow-collector");
 
+
     // Old simple collector set
     private final Map<Dpid, FlowStatsCollector> simpleCollectors = Maps.newConcurrentMap();
 
@@ -441,6 +443,8 @@
                         pushFlowMetrics(dpid, (OFFlowStatsReply) msg);
                     } else if (((OFStatsReply) msg).getStatsType() == OFStatsType.TABLE) {
                         pushTableStatistics(dpid, (OFTableStatsReply) msg);
+                    } else if (((OFStatsReply) msg).getStatsType() == OFStatsType.FLOW_LIGHTWEIGHT) {
+                        pushFlowLightWeightMetrics(dpid, (OFFlowLightweightStatsReply) msg);
                     }
                     break;
                 case BARRIER_REPLY:
@@ -653,6 +657,40 @@
             providerService.pushTableStatistics(did, tableStatsEntries);
         }
 
+        private void pushFlowLightWeightMetrics(Dpid dpid, OFFlowLightweightStatsReply replies) {
+
+            DeviceId did = DeviceId.deviceId(Dpid.uri(dpid));
+            NewAdaptiveFlowStatsCollector afsc = afsCollectors.get(dpid);
+            if (adaptiveFlowSampling && afsc != null)  {
+                List<FlowEntry> flowEntries = replies.getEntries().stream()
+                        .map(entry -> new FlowEntryBuilder(did, entry, driverService).withSetAfsc(afsc).build())
+                        .collect(Collectors.toList());
+
+                // Check that OFFlowStatsReply Xid is same with the one of OFFlowStatsRequest?
+                if (afsc.getFlowMissingXid() != NewAdaptiveFlowStatsCollector.NO_FLOW_MISSING_XID) {
+                    log.debug("OpenFlowRuleProvider:pushFlowMetrics, flowMissingXid={}, "
+                                    + "OFFlowStatsReply Xid={}, for {}",
+                            afsc.getFlowMissingXid(), replies.getXid(), dpid);
+                    if (afsc.getFlowMissingXid() == replies.getXid()) {
+                        // call entire flow stats update with flowMissing synchronization.
+                        // used existing pushFlowMetrics
+                        providerService.pushFlowMetrics(did, flowEntries);
+                    }
+                    // reset flowMissingXid to NO_FLOW_MISSING_XID
+                    afsc.setFlowMissingXid(NewAdaptiveFlowStatsCollector.NO_FLOW_MISSING_XID);
+                } else {
+                    // call individual flow stats update
+                    providerService.pushFlowMetricsWithoutFlowMissing(did, flowEntries);
+                }
+            } else {
+                List<FlowEntry> flowEntries = replies.getEntries().stream()
+                        .map(entry -> new FlowEntryBuilder(did, entry, driverService).build())
+                        .collect(Collectors.toList());
+                // call existing entire flow stats update with flowMissing synchronization
+                providerService.pushFlowMetrics(did, flowEntries);
+            }
+        }
+
         private TableStatisticsEntry buildTableStatistics(DeviceId deviceId,
                                                           OFTableStatsEntry ofEntry) {
             TableStatisticsEntry entry = null;
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java
index 083a74c..c16c34f 100644
--- a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java
@@ -16,6 +16,7 @@
 package org.onosproject.provider.of.flow.util;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.onlab.packet.EthType;
 import org.onlab.packet.Ip4Address;
 import org.onlab.packet.Ip4Prefix;
@@ -42,6 +43,8 @@
 import org.onosproject.net.flow.FlowEntry;
 import org.onosproject.net.flow.FlowEntry.FlowEntryState;
 import org.onosproject.net.flow.FlowRule;
+import org.onosproject.net.flow.StatTriggerField;
+import org.onosproject.net.flow.StatTriggerFlag;
 import org.onosproject.net.flow.TrafficSelector;
 import org.onosproject.net.flow.TrafficTreatment;
 import org.onosproject.net.flow.criteria.ExtensionSelectorType.ExtensionSelectorTypes;
@@ -49,11 +52,14 @@
 import org.onosproject.openflow.controller.ExtensionSelectorInterpreter;
 import org.onosproject.openflow.controller.ExtensionTreatmentInterpreter;
 import org.onosproject.provider.of.flow.impl.NewAdaptiveFlowStatsCollector;
+import org.projectfloodlight.openflow.protocol.OFFlowLightweightStatsEntry;
 import org.projectfloodlight.openflow.protocol.OFFlowMod;
 import org.projectfloodlight.openflow.protocol.OFFlowRemoved;
 import org.projectfloodlight.openflow.protocol.OFFlowStatsEntry;
 import org.projectfloodlight.openflow.protocol.OFMatchV3;
 import org.projectfloodlight.openflow.protocol.OFObject;
+import org.projectfloodlight.openflow.protocol.OFOxsList;
+import org.projectfloodlight.openflow.protocol.OFStatTriggerFlags;
 import org.projectfloodlight.openflow.protocol.OFVersion;
 import org.projectfloodlight.openflow.protocol.action.OFAction;
 import org.projectfloodlight.openflow.protocol.action.OFActionCircuit;
@@ -74,12 +80,14 @@
 import org.projectfloodlight.openflow.protocol.instruction.OFInstruction;
 import org.projectfloodlight.openflow.protocol.instruction.OFInstructionApplyActions;
 import org.projectfloodlight.openflow.protocol.instruction.OFInstructionGotoTable;
+import org.projectfloodlight.openflow.protocol.instruction.OFInstructionStatTrigger;
 import org.projectfloodlight.openflow.protocol.instruction.OFInstructionWriteActions;
 import org.projectfloodlight.openflow.protocol.instruction.OFInstructionWriteMetadata;
 import org.projectfloodlight.openflow.protocol.match.Match;
 import org.projectfloodlight.openflow.protocol.match.MatchField;
 import org.projectfloodlight.openflow.protocol.oxm.OFOxm;
 import org.projectfloodlight.openflow.protocol.oxm.OFOxmOchSigid;
+import org.projectfloodlight.openflow.protocol.oxs.OFOxs;
 import org.projectfloodlight.openflow.protocol.ver13.OFFactoryVer13;
 import org.projectfloodlight.openflow.types.CircuitSignalID;
 import org.projectfloodlight.openflow.types.IPv4Address;
@@ -97,9 +105,15 @@
 import org.slf4j.LoggerFactory;
 
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.onosproject.net.flow.StatTriggerField.*;
+import static org.onosproject.net.flow.StatTriggerField.IDLE_TIME;
+import static org.onosproject.net.flow.StatTriggerFlag.ONLY_FIRST;
+import static org.onosproject.net.flow.StatTriggerFlag.PERIODIC;
 import static org.onosproject.net.flow.criteria.Criteria.*;
 import static org.onosproject.net.flow.instructions.Instructions.modL0Lambda;
 import static org.onosproject.net.flow.instructions.Instructions.modL1OduSignalId;
@@ -111,6 +125,7 @@
     private final OFFlowStatsEntry stat;
     private final OFFlowRemoved removed;
     private final OFFlowMod flowMod;
+    private final OFFlowLightweightStatsEntry lightWeightStat;
 
     private final Match match;
 
@@ -120,7 +135,7 @@
 
     private final DeviceId deviceId;
 
-    public enum FlowType { STAT, REMOVED, MOD }
+    public enum FlowType { STAT, LIGHTWEIGHT_STAT, REMOVED, MOD }
 
     private final FlowType type;
 
@@ -140,6 +155,21 @@
         this.type = FlowType.STAT;
         this.driverService = driverService;
         this.afsc = null;
+        this.lightWeightStat = null;
+    }
+
+    public FlowEntryBuilder(DeviceId deviceId, OFFlowLightweightStatsEntry lightWeightStat,
+                            DriverService driverService) {
+        this.stat = null;
+        this.match = lightWeightStat.getMatch();
+        this.instructions = null;
+        this.deviceId = deviceId;
+        this.removed = null;
+        this.flowMod = null;
+        this.type = FlowType.LIGHTWEIGHT_STAT;
+        this.driverService = driverService;
+        this.afsc = null;
+        this.lightWeightStat = lightWeightStat;
     }
 
     public FlowEntryBuilder(DeviceId deviceId, OFFlowRemoved removed, DriverService driverService) {
@@ -152,6 +182,7 @@
         this.type = FlowType.REMOVED;
         this.driverService = driverService;
         this.afsc = null;
+        this.lightWeightStat = null;
     }
 
     public FlowEntryBuilder(DeviceId deviceId, OFFlowMod fm, DriverService driverService) {
@@ -164,6 +195,7 @@
         this.removed = null;
         this.driverService = driverService;
         this.afsc = null;
+        this.lightWeightStat = null;
     }
 
     public FlowEntryBuilder withSetAfsc(NewAdaptiveFlowStatsCollector afsc) {
@@ -172,83 +204,16 @@
     }
 
     public FlowEntry build(FlowEntryState... state) {
-        FlowRule.Builder builder;
         try {
             switch (this.type) {
                 case STAT:
-                    builder = DefaultFlowRule.builder()
-                            .forDevice(deviceId)
-                            .withSelector(buildSelector())
-                            .withTreatment(buildTreatment())
-                            .withPriority(stat.getPriority())
-                            .withIdleTimeout(stat.getIdleTimeout())
-                            .withCookie(stat.getCookie().getValue());
-                    if (stat.getVersion() != OFVersion.OF_10) {
-                        builder.forTable(stat.getTableId().getValue());
-                    }
-
-                    if (afsc != null) {
-                        FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(stat.getDurationSec());
-                        return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
-                                                    SECONDS.toNanos(stat.getDurationSec())
-                                                            + stat.getDurationNsec(), NANOSECONDS,
-                                                    liveType,
-                                                    stat.getPacketCount().getValue(),
-                                                    stat.getByteCount().getValue());
-                    } else {
-                        return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
-                                                    stat.getDurationSec(),
-                                                    stat.getPacketCount().getValue(),
-                                                    stat.getByteCount().getValue());
-                    }
-
+                    return createFlowEntryFromStat();
+                case LIGHTWEIGHT_STAT:
+                    return createFlowEntryFromLightweightStat();
                 case REMOVED:
-                    builder = DefaultFlowRule.builder()
-                            .forDevice(deviceId)
-                            .withSelector(buildSelector())
-                            .withPriority(removed.getPriority())
-                            .withIdleTimeout(removed.getIdleTimeout())
-                            .withCookie(removed.getCookie().getValue())
-                            .withReason(FlowRule.FlowRemoveReason.parseShort((short) removed.getReason().ordinal()));
-
-                    if (removed.getVersion() != OFVersion.OF_10) {
-                        builder.forTable(removed.getTableId().getValue());
-                    }
-
-                    if (afsc != null) {
-                        FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(removed.getDurationSec());
-                        return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
-                                                    SECONDS.toNanos(removed.getDurationSec())
-                                                            + removed.getDurationNsec(), NANOSECONDS,
-                                                    liveType,
-                                                    removed.getPacketCount().getValue(),
-                                                    removed.getByteCount().getValue());
-                    } else {
-                        return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
-                                                    removed.getDurationSec(),
-                                                    removed.getPacketCount().getValue(),
-                                                    removed.getByteCount().getValue());
-                    }
-
+                    return createFlowEntryForFlowRemoved();
                 case MOD:
-                    FlowEntryState flowState = state.length > 0 ? state[0] : FlowEntryState.FAILED;
-                    builder = DefaultFlowRule.builder()
-                            .forDevice(deviceId)
-                            .withSelector(buildSelector())
-                            .withTreatment(buildTreatment())
-                            .withPriority(flowMod.getPriority())
-                            .withIdleTimeout(flowMod.getIdleTimeout())
-                            .withCookie(flowMod.getCookie().getValue());
-                    if (flowMod.getVersion() != OFVersion.OF_10) {
-                        builder.forTable(flowMod.getTableId().getValue());
-                    }
-
-                    if (afsc != null) {
-                        FlowEntry.FlowLiveType liveType = FlowEntry.FlowLiveType.IMMEDIATE;
-                        return new DefaultFlowEntry(builder.build(), flowState, 0, liveType, 0, 0);
-                    } else {
-                        return new DefaultFlowEntry(builder.build(), flowState, 0, 0, 0);
-                    }
+                    return createFlowEntryForFlowMod(state);
                 default:
                     log.error("Unknown flow type : {}", this.type);
                     return null;
@@ -260,6 +225,143 @@
 
     }
 
+    private FlowEntry createFlowEntryFromStat() {
+
+        FlowRule.Builder builder = DefaultFlowRule.builder()
+                .forDevice(deviceId)
+                .withSelector(buildSelector())
+                .withTreatment(buildTreatment())
+                .withPriority(stat.getPriority())
+                .withIdleTimeout(stat.getIdleTimeout())
+                .withCookie(stat.getCookie().getValue());
+        if (stat.getVersion() != OFVersion.OF_10) {
+            builder.forTable(stat.getTableId().getValue());
+        }
+        if (stat.getVersion().getWireVersion() < OFVersion.OF_15.getWireVersion()) {
+            if (afsc != null) {
+                FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(stat.getDurationSec());
+                return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                        SECONDS.toNanos(stat.getDurationSec())
+                                + stat.getDurationNsec(), NANOSECONDS,
+                        liveType,
+                        stat.getPacketCount().getValue(),
+                        stat.getByteCount().getValue());
+            } else {
+                return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                        stat.getDurationSec(),
+                        stat.getPacketCount().getValue(),
+                        stat.getByteCount().getValue());
+            }
+        }
+        FlowStatParser statParser = new FlowStatParser(stat.getStats());
+        if (afsc != null && statParser.isDurationReceived()) {
+            FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(statParser.getDuration());
+            return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                    SECONDS.toNanos(statParser.getDuration())
+                            + SECONDS.toNanos(statParser.getDuration()), NANOSECONDS,
+                    liveType,
+                    statParser.getPacketCount(),
+                    statParser.getByteCount());
+        } else {
+            return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                    statParser.getDuration(),
+                    statParser.getPacketCount(),
+                    statParser.getByteCount());
+        }
+
+    }
+
+    private FlowEntry createFlowEntryForFlowMod(FlowEntryState ...state) {
+        FlowEntryState flowState = state.length > 0 ? state[0] : FlowEntryState.FAILED;
+        FlowRule.Builder builder = DefaultFlowRule.builder()
+                .forDevice(deviceId)
+                .withSelector(buildSelector())
+                .withTreatment(buildTreatment())
+                .withPriority(flowMod.getPriority())
+                .withIdleTimeout(flowMod.getIdleTimeout())
+                .withCookie(flowMod.getCookie().getValue());
+        if (flowMod.getVersion() != OFVersion.OF_10) {
+            builder.forTable(flowMod.getTableId().getValue());
+        }
+
+        if (afsc != null) {
+            FlowEntry.FlowLiveType liveType = FlowEntry.FlowLiveType.IMMEDIATE;
+            return new DefaultFlowEntry(builder.build(), flowState, 0, liveType, 0, 0);
+        } else {
+            return new DefaultFlowEntry(builder.build(), flowState, 0, 0, 0);
+        }
+    }
+
+    private FlowEntry createFlowEntryForFlowRemoved() {
+        FlowRule.Builder builder = DefaultFlowRule.builder()
+                .forDevice(deviceId)
+                .withSelector(buildSelector())
+                .withPriority(removed.getPriority())
+                .withIdleTimeout(removed.getIdleTimeout())
+                .withCookie(removed.getCookie().getValue())
+                .withReason(FlowRule.FlowRemoveReason.parseShort((short) removed.getReason().ordinal()));
+
+        if (removed.getVersion() != OFVersion.OF_10) {
+            builder.forTable(removed.getTableId().getValue());
+        }
+        if (removed.getVersion().getWireVersion() < OFVersion.OF_15.getWireVersion()) {
+            if (afsc != null) {
+                FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(removed.getDurationSec());
+                return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
+                        SECONDS.toNanos(removed.getDurationSec())
+                                + removed.getDurationNsec(), NANOSECONDS,
+                        liveType,
+                        removed.getPacketCount().getValue(),
+                        removed.getByteCount().getValue());
+            } else {
+                return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
+                        removed.getDurationSec(),
+                        removed.getPacketCount().getValue(),
+                        removed.getByteCount().getValue());
+            }
+        }
+        FlowStatParser statParser = new FlowStatParser(removed.getStats());
+        if (afsc != null && statParser.isDurationReceived()) {
+            FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(statParser.getDuration());
+            return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
+                    SECONDS.toNanos(statParser.getDuration())
+                            + SECONDS.toNanos(statParser.getDuration()), NANOSECONDS,
+                    liveType,
+                    statParser.getPacketCount(),
+                    statParser.getByteCount());
+        } else {
+            return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
+                    statParser.getDuration(),
+                    statParser.getPacketCount(),
+                    statParser.getByteCount());
+        }
+    }
+
+    private FlowEntry createFlowEntryFromLightweightStat() {
+        FlowRule.Builder builder = DefaultFlowRule.builder()
+                .forDevice(deviceId)
+                .withSelector(buildSelector())
+                .withPriority(lightWeightStat.getPriority())
+                .withIdleTimeout(0)
+                .withCookie(0);
+        FlowStatParser flowLightweightStatParser = new FlowStatParser(lightWeightStat.getStats());
+        builder.forTable(lightWeightStat.getTableId().getValue());
+        if (afsc != null && flowLightweightStatParser.isDurationReceived()) {
+            FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(flowLightweightStatParser.getDuration());
+            return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                    SECONDS.toNanos(flowLightweightStatParser.getDuration())
+                            + flowLightweightStatParser.getDuration(), NANOSECONDS,
+                    liveType,
+                    flowLightweightStatParser.getPacketCount(),
+                    flowLightweightStatParser.getByteCount());
+        } else {
+            return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                    flowLightweightStatParser.getDuration(),
+                    flowLightweightStatParser.getPacketCount(),
+                    flowLightweightStatParser.getByteCount());
+        }
+    }
+
     private List<OFInstruction> getInstructions(OFFlowMod entry) {
         switch (entry.getVersion()) {
             case OF_10:
@@ -321,6 +423,10 @@
                 case CLEAR_ACTIONS:
                     builder.wipeDeferred();
                     break;
+                case STAT_TRIGGER:
+                    OFInstructionStatTrigger statTrigger = (OFInstructionStatTrigger) in;
+                    buildStatTrigger(statTrigger.getThresholds(), statTrigger.getFlags(), builder);
+                    break;
                 case EXPERIMENTER:
                     break;
                 case METER:
@@ -333,6 +439,57 @@
         return builder.build();
     }
 
+    private TrafficTreatment.Builder buildStatTrigger(OFOxsList oxsList,
+                                                      Set<OFStatTriggerFlags> flagsSet,
+                                                      TrafficTreatment.Builder builder) {
+        Map<StatTriggerField, Long> statTriggerMap = Maps.newEnumMap(StatTriggerField.class);
+        for (OFOxs<?> ofOxs : oxsList) {
+            switch (ofOxs.getStatField().id) {
+                case DURATION:
+                    U64 durationType = (U64) ofOxs.getValue();
+                    statTriggerMap.put(DURATION, durationType.getValue());
+                    break;
+                case FLOW_COUNT:
+                    U32 flowCount = (U32) ofOxs.getValue();
+                    statTriggerMap.put(FLOW_COUNT, flowCount.getValue());
+                    break;
+                case PACKET_COUNT:
+                    U64 packetCount = (U64) ofOxs.getValue();
+                    statTriggerMap.put(PACKET_COUNT, packetCount.getValue());
+                    break;
+                case BYTE_COUNT:
+                    U64 byteCount = (U64) ofOxs.getValue();
+                    statTriggerMap.put(BYTE_COUNT, byteCount.getValue());
+                    break;
+                case IDLE_TIME:
+                    U64 idleTime = (U64) ofOxs.getValue();
+                    statTriggerMap.put(IDLE_TIME, idleTime.getValue());
+                    break;
+                default:
+                    log.warn("getStatField not supported {}", ofOxs.getStatField().id);
+                    break;
+            }
+        }
+        StatTriggerFlag flag = null;
+        for (OFStatTriggerFlags flags : flagsSet) {
+            switch (flags) {
+                case PERIODIC:
+                    flag = PERIODIC;
+                    break;
+                case ONLY_FIRST:
+                    flag = ONLY_FIRST;
+                    break;
+                default:
+                    log.warn("flag not supported {}", flags);
+                    break;
+            }
+        }
+        if (!statTriggerMap.isEmpty() && flag != null) {
+            builder.add(Instructions.statTrigger(statTriggerMap, flag));
+        }
+        return builder;
+    }
+
     /**
      * Configures traffic treatment builder with a given collection of actions.
      *
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowStatParser.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowStatParser.java
new file mode 100644
index 0000000..dfea390
--- /dev/null
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowStatParser.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2014-present Open Networking Foundation
+ *
+ * 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.
+ */
+package org.onosproject.provider.of.flow.util;
+
+import org.projectfloodlight.openflow.protocol.stat.Stat;
+import org.projectfloodlight.openflow.protocol.stat.StatField;
+import org.projectfloodlight.openflow.types.U32;
+import org.projectfloodlight.openflow.types.U64;
+
+/**
+ * FlowStatParser helps to parse OXS which is added in OPF 1.5.
+ */
+public final class FlowStatParser {
+    private final Stat stat;
+
+
+    private long duration;
+    private long idleTime;
+    private long flowCount;
+    private long packetCount;
+    private long byteCount;
+    private boolean isDurationReceived;
+
+    public FlowStatParser(Stat stat) {
+        this.stat = stat;
+        parseStats();
+    }
+
+    public Stat getStat() {
+        return stat;
+    }
+
+    private void parseStats() {
+        U64 durationOfValue = this.stat.get(StatField.DURATION);
+        U64 byteCountOfValue = this.stat.get(StatField.BYTE_COUNT);
+        U32 flowCountOfValue = this.stat.get(StatField.FLOW_COUNT);
+        U64 idleTimeOfValue = this.stat.get(StatField.IDLE_TIME);
+        U64 packetCountOfValue = this.stat.get(StatField.PACKET_COUNT);
+
+        isDurationReceived = durationOfValue != null;
+        duration = durationOfValue != null ? durationOfValue.getValue() : 0;
+        byteCount = byteCountOfValue != null ? byteCountOfValue.getValue() : 0;
+        idleTime = idleTimeOfValue != null ? idleTimeOfValue.getValue() : 0;
+        flowCount = flowCountOfValue != null ? flowCountOfValue.getValue() : 0;
+        packetCount = packetCountOfValue != null ? packetCountOfValue.getValue() : 0;
+    }
+
+
+    public long getByteCount() {
+        return byteCount;
+    }
+
+    public long getDuration() {
+        return duration;
+    }
+
+    public long getFlowCount() {
+        return flowCount;
+    }
+
+    public long getPacketCount() {
+        return packetCount;
+    }
+
+    public long getIdleTime() {
+        return idleTime;
+    }
+
+    public boolean isDurationReceived() {
+        return isDurationReceived;
+    }
+}
diff --git a/providers/openflow/meter/src/main/java/org/onosproject/provider/of/meter/impl/OpenFlowMeterProvider.java b/providers/openflow/meter/src/main/java/org/onosproject/provider/of/meter/impl/OpenFlowMeterProvider.java
index 3d0dacd..bcacb29 100644
--- a/providers/openflow/meter/src/main/java/org/onosproject/provider/of/meter/impl/OpenFlowMeterProvider.java
+++ b/providers/openflow/meter/src/main/java/org/onosproject/provider/of/meter/impl/OpenFlowMeterProvider.java
@@ -339,8 +339,9 @@
             meter.setLife(stat.getDurationSec());
             meter.setProcessedBytes(stat.getByteInCount().getValue());
             meter.setProcessedPackets(stat.getPacketInCount().getValue());
-            meter.setReferenceCount(stat.getFlowCount());
-
+            if (stat.getVersion().getWireVersion() < OFVersion.OF_15.getWireVersion()) {
+                meter.setReferenceCount(stat.getFlowCount());
+            }
             // marks the meter as seen on the dataplane
             pendingOperations.invalidate(stat.getMeterId());
             return meter;
diff --git a/providers/openflow/packet/src/main/java/org/onosproject/provider/of/packet/impl/OpenFlowPacketProvider.java b/providers/openflow/packet/src/main/java/org/onosproject/provider/of/packet/impl/OpenFlowPacketProvider.java
index 0eb0c04..c5e2a87 100644
--- a/providers/openflow/packet/src/main/java/org/onosproject/provider/of/packet/impl/OpenFlowPacketProvider.java
+++ b/providers/openflow/packet/src/main/java/org/onosproject/provider/of/packet/impl/OpenFlowPacketProvider.java
@@ -40,6 +40,7 @@
 import org.onosproject.openflow.controller.PacketListener;
 import org.projectfloodlight.openflow.protocol.OFPacketOut;
 import org.projectfloodlight.openflow.protocol.OFPortDesc;
+import org.projectfloodlight.openflow.protocol.OFVersion;
 import org.projectfloodlight.openflow.protocol.action.OFAction;
 import org.projectfloodlight.openflow.protocol.ver10.OFFactoryVer10;
 import org.projectfloodlight.openflow.types.OFBufferId;
@@ -136,12 +137,14 @@
                 .buildOutput()
                 .setPort(out)
                 .build();
-        return builder
-                .setBufferId(OFBufferId.NO_BUFFER)
-                .setInPort(OFPort.CONTROLLER)
+        builder.setBufferId(OFBufferId.NO_BUFFER)
                 .setActions(Collections.singletonList(act))
-                .setData(eth)
-                .build();
+                .setData(eth);
+        if (sw.factory().getVersion().getWireVersion() <= OFVersion.OF_14.getWireVersion()) {
+            builder.setInPort(OFPort.CONTROLLER);
+        }
+
+        return builder.build();
     }
 
     /**