Modified CPQD switch dirver to work with three switch scenario - taken from ONOS13 branch
Added ArpHandler for Segment Routing

Change-Id: I0b11ba0c298b6ed0cb16ed9a581ac61215000fad
diff --git a/src/main/java/net/onrc/onos/apps/segmentrouting/ArpHandler.java b/src/main/java/net/onrc/onos/apps/segmentrouting/ArpHandler.java
new file mode 100644
index 0000000..c2cab8a
--- /dev/null
+++ b/src/main/java/net/onrc/onos/apps/segmentrouting/ArpHandler.java
@@ -0,0 +1,463 @@
+/*******************************************************************************
+ * Copyright (c) 2014 Open Networking Laboratory.
+ * All rights reserved. This program and the accompanying materials
+ * are made available under the terms of the Apache License v2.0
+ * which accompanies this distribution, and is available at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ ******************************************************************************/
+
+package net.onrc.onos.apps.segmentrouting;
+
+import java.net.Inet4Address;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import net.floodlightcontroller.core.FloodlightContext;
+import net.floodlightcontroller.core.IFloodlightProviderService;
+import net.floodlightcontroller.core.IOFMessageListener;
+import net.floodlightcontroller.core.IOFSwitch;
+import net.floodlightcontroller.core.module.FloodlightModuleContext;
+import net.floodlightcontroller.core.module.FloodlightModuleException;
+import net.floodlightcontroller.core.module.IFloodlightModule;
+import net.floodlightcontroller.core.module.IFloodlightService;
+import net.floodlightcontroller.util.MACAddress;
+import net.onrc.onos.api.packet.IPacketListener;
+import net.onrc.onos.api.packet.IPacketService;
+import net.onrc.onos.core.flowprogrammer.IFlowPusherService;
+import net.onrc.onos.core.main.config.IConfigInfoService;
+import net.onrc.onos.core.packet.ARP;
+import net.onrc.onos.core.packet.Ethernet;
+import net.onrc.onos.core.packet.IPv4;
+import net.onrc.onos.core.topology.Host;
+import net.onrc.onos.core.topology.ITopologyService;
+import net.onrc.onos.core.topology.MutableTopology;
+import net.onrc.onos.core.topology.Port;
+import net.onrc.onos.core.topology.Switch;
+import net.onrc.onos.core.util.SwitchPort;
+
+import org.projectfloodlight.openflow.protocol.OFFactory;
+import org.projectfloodlight.openflow.protocol.OFMatchV3;
+import org.projectfloodlight.openflow.protocol.OFMessage;
+import org.projectfloodlight.openflow.protocol.OFOxmList;
+import org.projectfloodlight.openflow.protocol.OFType;
+import org.projectfloodlight.openflow.protocol.action.OFAction;
+import org.projectfloodlight.openflow.protocol.instruction.OFInstruction;
+import org.projectfloodlight.openflow.protocol.oxm.OFOxmEthDst;
+import org.projectfloodlight.openflow.protocol.oxm.OFOxmEthSrc;
+import org.projectfloodlight.openflow.protocol.oxm.OFOxmEthType;
+import org.projectfloodlight.openflow.protocol.oxm.OFOxmIpv4DstMasked;
+import org.projectfloodlight.openflow.types.EthType;
+import org.projectfloodlight.openflow.types.IPv4Address;
+import org.projectfloodlight.openflow.types.MacAddress;
+import org.projectfloodlight.openflow.types.OFBufferId;
+import org.projectfloodlight.openflow.types.OFPort;
+import org.projectfloodlight.openflow.types.TableId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.minlog.Log;
+
+/**
+ * Handling ARP requests to switches for Segment Routing.
+ * <p/>
+ * The module is for handling ARP requests to switches. It sends ARP response for any known
+ * hosts to the controllers.
+ * TODO: need to check the network config file for all hosts and packets
+ */
+public class ArpHandler implements IFloodlightModule, IOFMessageListener, IPacketListener  {
+
+    private static final Logger log = LoggerFactory
+            .getLogger(ArpHandler.class);
+
+    private IFloodlightProviderService floodlightProvider;
+    private IPacketService packetService;
+    private IFlowPusherService flowPusher;
+    private ITopologyService topologyService;
+    private MutableTopology mutableTopology;
+    private List<ArpEntry> arpEntries;
+
+    private static final short IDLE_TIMEOUT = 0;
+    private static final short HARD_TIMEOUT = 0;
+
+    private static final int TABLE_VLAN = 0;
+    private static final int TABLE_TMAC = 1;
+    private static final int TABLE_IPv4_UNICAST = 2;
+    private static final int TABLE_MPLS = 3;
+    private static final int TABLE_META = 4;
+    private static final int TABLE_ACL = 5;
+
+    private static final short MAX_PRIORITY = (short) 0xffff;
+    private static final short SLASH_24_PRIORITY = (short) 0xfff0;
+    private static final short SLASH_16_PRIORITY = (short) 0xff00;
+    private static final short SLASH_8_PRIORITY = (short) 0xf000;
+    private static final short MIN_PRIORITY = 0x0;
+
+
+    @Override
+    public Collection<Class<? extends IFloodlightService>> getModuleServices() {
+        return null;
+    }
+
+    @Override
+    public Map<Class<? extends IFloodlightService>, IFloodlightService> getServiceImpls() {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Collection<Class<? extends IFloodlightService>> getModuleDependencies() {
+        Collection<Class<? extends IFloodlightService>> l = new ArrayList<Class<? extends IFloodlightService>>();
+        l.add(IFloodlightProviderService.class);
+        l.add(IConfigInfoService.class);
+        l.add(ITopologyService.class);
+        l.add(IPacketService.class);
+        l.add(IFlowPusherService.class);
+        l.add(ITopologyService.class);
+
+        return l;
+    }
+
+    @Override
+    public void init(FloodlightModuleContext context) throws FloodlightModuleException {
+        this.floodlightProvider = context.getServiceImpl(IFloodlightProviderService.class);
+        this.packetService = context.getServiceImpl(IPacketService.class);
+        this.flowPusher = context.getServiceImpl(IFlowPusherService.class);
+        this.topologyService = context.getServiceImpl(ITopologyService.class);
+
+        Log.debug("Arp Handler is initialized");
+
+    }
+
+    @Override
+    public String getName() {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public boolean isCallbackOrderingPrereq(OFType type, String name) {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public boolean isCallbackOrderingPostreq(OFType type, String name) {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public net.floodlightcontroller.core.IListener.Command receive(IOFSwitch sw,
+            OFMessage msg, FloodlightContext cntx) {
+
+        return Command.CONTINUE;
+    }
+
+    @Override
+    public void startUp(FloodlightModuleContext context) throws FloodlightModuleException {
+
+        packetService.registerPacketListener(this);
+        floodlightProvider.addOFMessageListener(OFType.PACKET_IN, this);
+        mutableTopology = topologyService.getTopology();
+        arpEntries = new ArrayList<ArpEntry>();
+    }
+
+    @Override
+    public void receive(Switch sw, Port inPort, Ethernet payload) {
+        log.debug("Received a packet {} from sw {} ", payload.toString(), sw.getDpid());
+
+        if (payload.getEtherType() == Ethernet.TYPE_ARP) {
+
+            ARP arp = (ARP)payload.getPayload();
+            updateArpCache(arp);
+
+            if (arp.getOpCode() == ARP.OP_REQUEST) {
+
+                handleArpRequest(sw, inPort, arp);
+            }
+
+        }
+        else if (payload.getEtherType() == Ethernet.TYPE_IPV4) {
+
+            IPv4 ipv4 = (IPv4)payload.getPayload();
+            if (ipv4.getProtocol() == IPv4.PROTOCOL_ICMP) {
+
+                addRouteToHost(sw, ipv4);
+
+            }
+
+        }
+
+    }
+
+    /**
+     * Add routing rules to forward packets to known hosts
+     *
+     * @param sw Switch
+     * @param hostIp Host IP address to forwards packets to
+     */
+    private void addRouteToHost(Switch sw, IPv4 hostIp) {
+
+
+        IOFSwitch ofSwitch = floodlightProvider.getMasterSwitch(sw.getDpid().value());
+        OFFactory factory = ofSwitch.getFactory();
+        int destinationAddress = hostIp.getDestinationAddress();
+        // Check APR entries
+        byte[] destinationMacAddress = getMacAddressFromIpAddress(destinationAddress);
+
+        // Check TopologyService
+        for (Host host: mutableTopology.getHosts()) {
+            IPv4Address hostIpAddress = IPv4Address.of(host.getIpAddress());
+            if (hostIpAddress != null && hostIpAddress.getInt() == destinationAddress) {
+                destinationMacAddress = host.getMacAddress().toBytes();
+            }
+        }
+
+        // If MAC address is not known to the host, just return
+        if (destinationMacAddress == null)
+            return;
+
+        OFOxmEthType ethTypeIp = factory.oxms()
+                .ethType(EthType.IPv4);
+        OFOxmIpv4DstMasked ipPrefix = factory.oxms()
+                .ipv4DstMasked(
+                        IPv4Address.of(destinationAddress),
+                        IPv4Address.NO_MASK); // host addr should be /32
+        OFOxmList oxmListSlash32 = OFOxmList.of(ethTypeIp, ipPrefix);
+        OFMatchV3 match = factory.buildMatchV3()
+                .setOxmList(oxmListSlash32).build();
+        OFAction setDmac = null;
+        OFOxmEthDst dmac = factory.oxms()
+                .ethDst(MacAddress.of(destinationMacAddress));
+        setDmac = factory.actions().buildSetField()
+                .setField(dmac).build();
+
+        OFAction decTtl = factory.actions().decNwTtl();
+
+        // Set the source MAC address with the switch MAC address
+        String switchMacAddress = sw.getStringAttribute("routerMac");
+        OFOxmEthSrc srcAddr = factory.oxms().ethSrc(MacAddress.of(switchMacAddress));
+        OFAction setSA = factory.actions().buildSetField()
+                .setField(srcAddr).build();
+
+        List<OFAction> actionList = new ArrayList<OFAction>();
+        actionList.add(setDmac);
+        actionList.add(decTtl);
+        actionList.add(setSA);
+
+
+        /* TODO : need to check the config file for all packets
+        String subnets = sw.getStringAttribute("subnets");
+        try {
+            JSONArray arry = new JSONArray(subnets);
+            for (int i = 0; i < arry.length(); i++) {
+                String subnetIp = (String) arry.getJSONObject(i).get("subnetIp");
+                int portNo = (int) arry.getJSONObject(i).get("portNo");
+
+                if (netMatch(subnetIp, IPv4Address.of(hostIp.getDestinationAddress()).toString())) {
+                    OFAction out = factory.actions().buildOutput()
+                            .setPort(OFPort.of(portNo)).build();
+                    actionList.add(out);
+                }
+            }
+        } catch (JSONException e) {
+            // TODO Auto-generated catch block
+            e.printStackTrace();
+        }
+        */
+
+        // Set output port
+        net.onrc.onos.core.topology.Host host = mutableTopology.getHostByMac(MACAddress.valueOf(destinationMacAddress));
+        if (host != null) {
+            for (Port port: host.getAttachmentPoints()) {
+                OFAction out = factory.actions().buildOutput()
+                                .setPort(OFPort.of(port.getPortNumber().shortValue())).build();
+                actionList.add(out);
+            }
+        }
+
+        OFInstruction writeInstr = factory.instructions().buildWriteActions()
+                .setActions(actionList).build();
+
+        List<OFInstruction> instructions = new ArrayList<OFInstruction>();
+        instructions.add(writeInstr);
+
+        OFMessage myIpEntry = factory.buildFlowAdd()
+                .setTableId(TableId.of(TABLE_IPv4_UNICAST))
+                .setMatch(match)
+                .setInstructions(instructions)
+                .setPriority(MAX_PRIORITY)
+                .setBufferId(OFBufferId.NO_BUFFER)
+                .setIdleTimeout(0)
+                .setHardTimeout(0)
+                //.setXid(getNextTransactionId())
+                .build();
+
+        log.debug("Sending 'Routing information' OF message to the switch {}.", sw.getDpid().toString());
+
+        flowPusher.add(sw.getDpid(), myIpEntry);
+
+
+    }
+
+    /**
+     * Send an ARP response for the ARP request to the known switches
+     *
+     * @param sw Switch
+     * @param inPort port to send ARP response to
+     * @param arpRequest ARP request packet to handle
+     */
+    private void handleArpRequest(Switch sw, Port inPort, ARP arpRequest) {
+
+        String switchIpAddressSlash = sw.getStringAttribute("routerIp");
+        String switchMacAddressStr = sw.getStringAttribute("routerMac");
+        if (switchIpAddressSlash != null && switchMacAddressStr != null) {
+
+            String switchIpAddressStr = switchIpAddressSlash.substring(0, switchIpAddressSlash.indexOf('/'));
+            IPv4Address switchIpAddress = IPv4Address.of(switchIpAddressStr);
+            IPv4Address targetProtocolAddress = IPv4Address.of(arpRequest.getTargetProtocolAddress());
+            if (targetProtocolAddress.equals(switchIpAddress)) {
+                MACAddress targetMac = MACAddress.valueOf(switchMacAddressStr);
+
+                ARP arpReply = new ARP();
+                arpReply.setHardwareType(ARP.HW_TYPE_ETHERNET)
+                        .setProtocolType(ARP.PROTO_TYPE_IP)
+                        .setHardwareAddressLength(
+                                (byte) Ethernet.DATALAYER_ADDRESS_LENGTH)
+                        .setProtocolAddressLength((byte) IPv4.ADDRESS_LENGTH)
+                        .setOpCode(ARP.OP_REPLY)
+                        .setSenderHardwareAddress(targetMac.toBytes())
+                        .setSenderProtocolAddress(arpRequest.getTargetProtocolAddress())
+                        .setTargetHardwareAddress(arpRequest.getSenderHardwareAddress())
+                        .setTargetProtocolAddress(arpRequest.getSenderProtocolAddress());
+
+                Ethernet eth = new Ethernet();
+                eth.setDestinationMACAddress(arpRequest.getSenderHardwareAddress())
+                        .setSourceMACAddress(targetMac.toBytes())
+                        .setEtherType(Ethernet.TYPE_ARP).setPayload(arpReply);
+
+                packetService.sendPacket(eth, new SwitchPort(sw.getDpid(), inPort.getPortNumber()));
+            }
+        }
+    }
+
+    /**
+     * Update ARP Cache using ARP packets
+     * It is used to set destination MAC address to forward packets to known hosts.
+     * But, it will be replace with Host information of Topology service later.
+     *
+     * @param arp APR packets to use for updating ARP entries
+     */
+    private void updateArpCache(ARP arp) {
+
+        ArpEntry arpEntry = new ArpEntry(arp.getSenderHardwareAddress(), arp.getSenderProtocolAddress());
+        // TODO: Need to check the duplication
+        arpEntries.add(arpEntry);
+    }
+
+    /**
+     * Temporary class to to keep ARP entry
+     *
+     */
+
+    private class ArpEntry {
+
+        byte[] targetMacAddress;
+        byte[] targetIpAddress;
+
+        private ArpEntry(byte[] macAddress, byte[] ipAddress) {
+            this.targetMacAddress = macAddress;
+            this.targetIpAddress = ipAddress;
+        }
+
+    }
+
+    /**
+     * Get MAC address to known hosts
+     *
+     * @param destinationAddress IP address to get MAC address
+     * @return MAC Address to given IP address
+     */
+    private byte[] getMacAddressFromIpAddress(int destinationAddress) {
+
+        // Can't we get the host IP address from the TopologyService ??
+
+        Iterator<ArpEntry> iterator = arpEntries.iterator();
+
+        IPv4Address ipAddress = IPv4Address.of(destinationAddress);
+        byte[] ipAddressInByte = ipAddress.getBytes();
+
+        while (iterator.hasNext() ) {
+            ArpEntry arpEntry = iterator.next();
+            byte[] address = arpEntry.targetIpAddress;
+
+            IPv4Address a = IPv4Address.of(address);
+            IPv4Address b = IPv4Address.of(ipAddressInByte);
+
+            if ( a.equals(b)) {
+                log.debug("Found an arp entry");
+                return arpEntry.targetMacAddress;
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * The function checks if given IP matches to the given subnet mask
+     *
+     * @param addr - subnet address to match
+     * @param addr1 - IP address to check
+     * @return true if the IP address matches to the subnet, otherwise false
+     */
+
+    public static boolean netMatch(String addr, String addr1){ //addr is subnet address and addr1 is ip address. Function will return true, if addr1 is within addr(subnet)
+
+        String[] parts = addr.split("/");
+        String ip = parts[0];
+        int prefix;
+
+        if (parts.length < 2) {
+            prefix = 0;
+        } else {
+            prefix = Integer.parseInt(parts[1]);
+        }
+
+        Inet4Address a =null;
+        Inet4Address a1 =null;
+        try {
+            a = (Inet4Address) InetAddress.getByName(ip);
+            a1 = (Inet4Address) InetAddress.getByName(addr1);
+        } catch (UnknownHostException e){}
+
+        byte[] b = a.getAddress();
+        int ipInt = ((b[0] & 0xFF) << 24) |
+                         ((b[1] & 0xFF) << 16) |
+                         ((b[2] & 0xFF) << 8)  |
+                         ((b[3] & 0xFF) << 0);
+
+        byte[] b1 = a1.getAddress();
+        int ipInt1 = ((b1[0] & 0xFF) << 24) |
+                         ((b1[1] & 0xFF) << 16) |
+                         ((b1[2] & 0xFF) << 8)  |
+                         ((b1[3] & 0xFF) << 0);
+
+        int mask = ~((1 << (32 - prefix)) - 1);
+
+        if ((ipInt & mask) == (ipInt1 & mask)) {
+            return true;
+        }
+        else {
+            return false;
+        }
+}
+
+
+
+
+}
diff --git a/src/main/java/net/onrc/onos/core/drivermanager/OFSwitchImplCPqD13.java b/src/main/java/net/onrc/onos/core/drivermanager/OFSwitchImplCPqD13.java
index 8d7aecb..24a326b 100644
--- a/src/main/java/net/onrc/onos/core/drivermanager/OFSwitchImplCPqD13.java
+++ b/src/main/java/net/onrc/onos/core/drivermanager/OFSwitchImplCPqD13.java
@@ -34,7 +34,6 @@
 import org.projectfloodlight.openflow.protocol.oxm.OFOxmEthType;
 import org.projectfloodlight.openflow.protocol.oxm.OFOxmInPort;
 import org.projectfloodlight.openflow.protocol.oxm.OFOxmIpv4DstMasked;
-import org.projectfloodlight.openflow.protocol.oxm.OFOxmMetadataMasked;
 import org.projectfloodlight.openflow.protocol.oxm.OFOxmMplsLabel;
 import org.projectfloodlight.openflow.protocol.oxm.OFOxmVlanVid;
 import org.projectfloodlight.openflow.types.EthType;
@@ -42,7 +41,6 @@
 import org.projectfloodlight.openflow.types.MacAddress;
 import org.projectfloodlight.openflow.types.OFBufferId;
 import org.projectfloodlight.openflow.types.OFGroup;
-import org.projectfloodlight.openflow.types.OFMetadata;
 import org.projectfloodlight.openflow.types.OFPort;
 import org.projectfloodlight.openflow.types.OFVlanVidMatch;
 import org.projectfloodlight.openflow.types.TableId;
@@ -79,7 +77,7 @@
 
     private static final int TABLE_VLAN = 0;
     private static final int TABLE_TMAC = 1;
-    private static final int TABLE_IPV4_UNICAST = 2;
+    private static final int TABLE_IPv4_UNICAST = 2;
     private static final int TABLE_MPLS = 3;
     private static final int TABLE_META = 4;
     private static final int TABLE_ACL = 5;
@@ -125,41 +123,31 @@
         if (!usePipeline13) {
             // Send packet-in to controller if a packet misses the first table
             populateTableMissEntry(0, true, false, false, 0);
-        } // else {
-          // configureSwitch();
-        // }
+        } else {
+           configureSwitch();
+        }
         sendBarrier(true);
     }
 
+
     @Override
     public boolean isDriverHandshakeComplete() {
-        if (!startDriverHandshakeCalled) {
+        if (!startDriverHandshakeCalled)
             throw new SwitchDriverSubHandshakeNotStarted();
-        }
         return driverHandshakeComplete.get();
     }
 
     @Override
     public void processDriverHandshakeMessage(OFMessage m) {
-        if (!startDriverHandshakeCalled) {
+        if (!startDriverHandshakeCalled)
             throw new SwitchDriverSubHandshakeNotStarted();
-        }
-        if (driverHandshakeComplete.get()) {
+        if (driverHandshakeComplete.get())
             throw new SwitchDriverSubHandshakeCompleted(m);
-        }
-
-        if (!startDriverHandshakeCalled) {
-            throw new SwitchDriverSubHandshakeNotStarted();
-        }
-        if (driverHandshakeComplete.get()) {
-            throw new SwitchDriverSubHandshakeCompleted(m);
-        }
 
         switch (m.getType()) {
         case BARRIER_REPLY:
-            if (m.getXid() == barrierXidToWaitFor) {
+            if (m.getXid() == barrierXidToWaitFor)
                 driverHandshakeComplete.set(true);
-            }
             break;
 
         case ERROR:
@@ -254,11 +242,11 @@
 
     private void decodeAsyncGetReply(OFAsyncGetReply rep) {
         long frm = rep.getFlowRemovedMaskEqualMaster();
-        //long frs = rep.getFlowRemovedMaskSlave();
+        long frs = rep.getFlowRemovedMaskSlave();
         long pim = rep.getPacketInMaskEqualMaster();
-        //long pis = rep.getPacketInMaskSlave();
+        long pis = rep.getPacketInMaskSlave();
         long psm = rep.getPortStatusMaskEqualMaster();
-        //long pss = rep.getPortStatusMaskSlave();
+        long pss = rep.getPortStatusMaskSlave();
 
         if (role == Role.MASTER || role == Role.EQUAL) { // should separate
             log.info("FRM:{}", HexString.toHexString((frm & TEST_FLOW_REMOVED_MASK)));
@@ -306,7 +294,7 @@
                         .setPort(p.getPortNo()).build();
                 OFAction popVlan = factory.actions().popVlan();
                 List<OFAction> actions = new ArrayList<OFAction>();
-                actions.add(popVlan);
+                // actions.add(popVlan);
                 actions.add(out);
                 OFBucket bucket = factory.buildBucket()
                         .setActions(actions).build();
@@ -336,12 +324,10 @@
     }
 
     private MacAddress getRouterMacAddr() {
-        if (getId() == 0x3) {
+        if (getId() == 0x3)
             return MacAddress.of("00:00:07:07:07:80"); // router mac
-        }
-        if (getId() == 0x1) {
+        if (getId() == 0x1)
             return MacAddress.of("00:00:01:01:01:80");
-        }
         // switch 0x2
         return MacAddress.of("00:00:02:02:02:80");
     }
@@ -401,15 +387,14 @@
             List<OFAction> actions = new ArrayList<OFAction>();
             actions.add(decTtl); // decrement the IP TTL/do-checksum/check TTL
                                  // and MTU
-            actions.add(setVlan); // set the vlan-id of the exit-port (and
-                                  // l2group)
+            // actions.add(setVlan); // set the vlan-id of the exit-port (and
+            // l2group)
             actions.add(setSA); // set this routers mac address
             // make L3Unicast group setDA for known (configured) ports
             // that connect to other routers
             OFAction setDA = getDestAction(portnum);
-            if (setDA != null) {
+            if (setDA != null)
                 actions.add(setDA);
-            }
             actions.add(group);
 
             OFBucket bucket = factory.buildBucket()
@@ -458,7 +443,8 @@
                 List<OFAction> actions = new ArrayList<OFAction>();
                 actions.add(decMplsTtl); // decrement the MPLS
                                          // TTL/do-checksum/check TTL and MTU
-                actions.add(setVlan); // set the vlan-id of the exit-port (and
+                // actions.add(setVlan); // set the vlan-id of the exit-port
+                // (and
                                       // l2group)
                 actions.add(setSA); // set this routers mac address
                 actions.add(setDA);
@@ -483,31 +469,31 @@
      *
      * OFGroup group47 = OFGroup.of(47);
         OFAction outgroup1 = factory.actions()
-                        .buildGroup()
-                        .setGroup(group61)
-                        .build();
+                .buildGroup()
+                .setGroup(group61)
+                .build();
         OFBucket buc47_1 = factory.buildBucket()
-                        .setWeight(1)
-                        .setActions(Collections.singletonList(outgroup1))
-                        .build();
+                .setWeight(1)
+                .setActions(Collections.singletonList(outgroup1))
+                .build();
         OFAction outgroup2 = factory.actions()
-                        .buildGroup()
-                        .setGroup(group62)
-                        .build();
+                .buildGroup()
+                .setGroup(group62)
+                .build();
         OFBucket buc47_2 = factory.buildBucket()
-                        .setWeight(1)
-                        .setActions(Collections.singletonList(outgroup2))
-                        .build();
+                .setWeight(1)
+                .setActions(Collections.singletonList(outgroup2))
+                .build();
         List<OFBucket> buckets47 = new ArrayList<OFBucket>();
         buckets47.add(buc47_1);
         buckets47.add(buc47_2);
         OFMessage gmS12 = factory.buildGroupAdd()
-                        .setGroup(group47)
-                        .setBuckets(buckets47)
-                        .setGroupType(OFGroupType.SELECT)
-                        .setXid(getNextTransactionId())
-                        .build();
-        write(gmS12, null);     */
+                .setGroup(group47)
+                .setBuckets(buckets47)
+                .setGroupType(OFGroupType.SELECT)
+                .setXid(getNextTransactionId())
+                .build();
+        write(gmS12, null); */
 
     private void processStatsReply(OFStatsReply sr) {
         switch (sr.getStatsType()) {
@@ -574,7 +560,7 @@
                 OFInstruction gotoTbl = factory.instructions().buildGotoTable()
                         .setTableId(TableId.of(TABLE_TMAC)).build();
                 List<OFInstruction> instructions = new ArrayList<OFInstruction>();
-                instructions.add(appAction);
+                // instructions.add(appAction);
                 instructions.add(gotoTbl);
                 OFMessage flowEntry = factory.buildFlowAdd()
                         .setTableId(TableId.of(TABLE_VLAN))
@@ -603,7 +589,7 @@
         OFMatchV3 matchIp = factory.buildMatchV3()
                 .setOxmList(oxmListIp).build();
         OFInstruction gotoTblIp = factory.instructions().buildGotoTable()
-                .setTableId(TableId.of(TABLE_IPV4_UNICAST)).build();
+                .setTableId(TableId.of(TABLE_IPv4_UNICAST)).build();
         List<OFInstruction> instructionsIp = Collections.singletonList(gotoTblIp);
         OFMessage ipEntry = factory.buildFlowAdd()
                 .setTableId(TableId.of(TABLE_TMAC))
@@ -672,16 +658,15 @@
             subnetIps.add("10.0.3.0");
             subnetIps.add("10.0.1.0");
         }
-        // TODO needed?
-        //if (getId() == 0x2) {
-        //}
+        if (getId() == 0x2) {
+        }
         if (getId() == 0x3) {
             subnetIps.add("7.7.7.0");
         }
         return subnetIps;
     }
 
-    private static class RouteEntry {
+    private class RouteEntry {
         String prefix;
         String mask;
         int nextHopPort;
@@ -712,7 +697,7 @@
                     .add(new RouteEntry("192.168.0.3", "255.255.255.255", 6, 103));
             routerNextHopIps.add(new RouteEntry("7.7.7.0", "255.255.255.0", 6, 103));
         }
-        //if (getId() == 0x2) {
+        if (getId() == 0x2) {
             /* These are required for normal IP routing without labels.
             routerNextHopIps.add(new RouteEntry("192.168.0.1","255.255.255.255",1));
             routerNextHopIps.add(new RouteEntry("192.168.0.3","255.255.255.255",2));
@@ -720,7 +705,7 @@
             routerNextHopIps.add(new RouteEntry("10.0.2.0","255.255.255.0",1));
             routerNextHopIps.add(new RouteEntry("10.0.3.0","255.255.255.0",1));
             routerNextHopIps.add(new RouteEntry("7.7.7.0","255.255.255.0",2));*/
-        //}
+        }
         if (getId() == 0x3) {
             routerNextHopIps
                     .add(new RouteEntry("192.168.0.2", "255.255.255.255", 2, 102));
@@ -737,12 +722,12 @@
     private List<RouteEntry> getHostNextHopIps() {
         List<RouteEntry> hostNextHopIps = new ArrayList<RouteEntry>();
         if (getId() == 0x1) {
+            //hostNextHopIps.add(new RouteEntry("10.0.1.1", 1, "00:00:00:00:01:01")); // Just for Test - SSH
             hostNextHopIps.add(new RouteEntry("10.0.2.1", 4, "00:00:00:00:02:01"));
             hostNextHopIps.add(new RouteEntry("10.0.3.1", 5, "00:00:00:00:03:01"));
         }
-        // TODO needed?
-        //if (getId() == 0x2) {
-        //}
+        if (getId() == 0x2) {
+        }
         if (getId() == 0x3) {
             hostNextHopIps.add(new RouteEntry("7.7.7.7", 1, "00:00:07:07:07:07"));
         }
@@ -750,14 +735,14 @@
     }
 
     private void populateIpTable() throws IOException {
-        populateMyIps();
-        populateMySubnets();
+        // populateMyIps();
+        // populateMySubnets();
         populateRoutes();
         populateHostRoutes();
 
         // match for everything else to send to ACL table. Essentially
         // the table miss flow entry
-        populateTableMissEntry(TABLE_IPV4_UNICAST, false, true,
+        populateTableMissEntry(TABLE_IPv4_UNICAST, false, true,
                 true, TABLE_ACL);
     }
 
@@ -785,7 +770,7 @@
             instructions.add(writeInstr);
             instructions.add(gotoInstr);
             OFMessage myIpEntry = factory.buildFlowAdd()
-                    .setTableId(TableId.of(TABLE_IPV4_UNICAST))
+                    .setTableId(TableId.of(TABLE_IPv4_UNICAST))
                     .setMatch(match)
                     .setInstructions(instructions)
                     .setPriority(MAX_PRIORITY) // highest priority for exact
@@ -828,7 +813,7 @@
             instructions.add(writeInstr);
             instructions.add(gotoInstr);
             OFMessage myIpEntry = factory.buildFlowAdd()
-                    .setTableId(TableId.of(TABLE_IPV4_UNICAST))
+                    .setTableId(TableId.of(TABLE_IPv4_UNICAST))
                     .setMatch(match)
                     .setInstructions(instructions)
                     .setPriority(SLASH_24_PRIORITY)
@@ -877,47 +862,44 @@
             // OFAction setBos =
             // factory.actions().buildSetField().setField(bos).build();
 
-            /*
-            writeActions.add(pushlabel);  // need to be apply actions so can be
-            writeActions.add(copyTtlOut); // matched in pseudo-table
-            //writeActions.add(setlabelid); // bad support in cpqd
-            //writeActions.add(setBos); no support in loxigen
-            */
-
-            List<OFAction> applyActions = new ArrayList<OFAction>();
-            applyActions.add(pushlabel);
-            applyActions.add(copyTtlOut);
-            OFInstruction applyInstr = factory.instructions().buildApplyActions()
-                    .setActions(applyActions).build();
             List<OFAction> writeActions = new ArrayList<OFAction>();
+            writeActions.add(pushlabel);
+            writeActions.add(copyTtlOut);
+            writeActions.add(setlabelid);
+            // writeActions.add(setBos); no support in loxigen
+
+            // List<OFAction> applyActions = new ArrayList<OFAction>();
+            // applyActions.add(pushlabel);
+            // applyActions.add(copyTtlOut);
+            // OFInstruction applyInstr =
+            // factory.instructions().buildApplyActions()
+            // .setActions(applyActions).build();
             writeActions.add(outg); // group will decr mpls-ttl, set mac-sa/da,
                                     // vlan
             OFInstruction writeInstr = factory.instructions().buildWriteActions()
                     .setActions(writeActions).build();
+            OFInstruction gotoInstr = factory.instructions().buildGotoTable()
+                    .setTableId(TableId.of(TABLE_ACL)).build();
+            List<OFInstruction> instructions = new ArrayList<OFInstruction>();
+            instructions.add(writeInstr);
 
             // necessary to match in pseudo-table to overcome cpqd 1.3 flaw
-            OFInstruction writeMeta = factory.instructions().buildWriteMetadata()
+            /*OFInstruction writeMeta = factory.instructions().buildWriteMetadata()
                     .setMetadata(U64.of(routerNextHopIps.get(i).label))
                     .setMetadataMask(METADATA_MASK).build();
-            /*OFInstruction gotoInstr = factory.instructions().buildGotoTable()
-                        .setTableId(TableId.of(TABLE_ACL)).build();*/
             OFInstruction gotoInstr = factory.instructions().buildGotoTable()
-                    .setTableId(TableId.of(TABLE_META)).build();
-            List<OFInstruction> instructions = new ArrayList<OFInstruction>();
-            instructions.add(applyInstr);
-            // instructions.add(writeInstr);// cannot write here - causes switch
-            // to crash
-            instructions.add(writeMeta);
-            instructions.add(gotoInstr);
+                    .setTableId(TableId.of(TABLE_META)).build();*/
+            /*instructions.add(applyInstr);
+            instructions.add(writeMeta);*/
 
+            instructions.add(gotoInstr);
             int priority = -1;
-            if (routerNextHopIps.get(i).mask.equals("255.255.255.255")) {
+            if (routerNextHopIps.get(i).mask.equals("255.255.255.255"))
                 priority = MAX_PRIORITY;
-            } else {
+            else
                 priority = SLASH_24_PRIORITY;
-            }
             OFMessage myIpEntry = factory.buildFlowAdd()
-                    .setTableId(TableId.of(TABLE_IPV4_UNICAST))
+                    .setTableId(TableId.of(TABLE_IPv4_UNICAST))
                     .setMatch(match)
                     .setInstructions(instructions)
                     .setPriority(priority)
@@ -931,7 +913,7 @@
             // need to also handle psuedo-table entries to match-metadata and
             // set mpls
             // label-id
-            OFOxmEthType ethTypeMpls = factory.oxms()
+            /*OFOxmEthType ethTypeMpls = factory.oxms()
                     .ethType(EthType.MPLS_UNICAST);
             OFOxmMetadataMasked meta = factory.oxms()
                     .metadataMasked(
@@ -944,7 +926,7 @@
             writeActions2.add(setlabelid);
             OFAction outg2 = factory.actions().buildGroup()
                     .setGroup(OFGroup.of(routerNextHopIps.get(i).nextHopPort |
-                            (192 << VLAN_ID_OFFSET)))
+                                            (192 << VLAN_ID_OFFSET)))
                     .build();
             writeActions2.add(outg2);
             OFInstruction writeInstr2 = factory.instructions().buildWriteActions()
@@ -955,23 +937,23 @@
             // unfortunately have to apply this action too
             OFInstruction applyInstr2 = factory.instructions().buildApplyActions()
                     .setActions(writeActions2).build();
-            instructions2.add(applyInstr2);
+            instructions2.add(applyInstr2); */
             // instructions2.add(writeInstr2);
             // instructions2.add(gotoInstr2);
 
             /*OFMatchV3 match3 = factory.buildMatchV3()
-                        .setOxmList(OFOxmList.of(meta)).build();
+                    .setOxmList(OFOxmList.of(meta)).build();
             OFInstruction clearInstruction = factory.instructions().clearActions();
             List<OFInstruction> instructions3 = new ArrayList<OFInstruction>();
             OFAction outc = factory.actions().buildOutput()
-                        .setPort(OFPort.CONTROLLER).setMaxLen(OFPCML_NO_BUFFER)
-                        .build();
+                    .setPort(OFPort.CONTROLLER).setMaxLen(OFPCML_NO_BUFFER)
+                    .build();
             OFInstruction writec = factory.instructions()
-                        .writeActions(Collections.singletonList(outc));
+                    .writeActions(Collections.singletonList(outc));
             instructions3.add(clearInstruction);
             instructions3.add(writec);
             instructions3.add(gotoInstr2); */
-            OFMessage myMetaEntry = factory.buildFlowAdd()
+            /*OFMessage myMetaEntry = factory.buildFlowAdd()
                     .setTableId(TableId.of(TABLE_META))
                     .setMatch(matchMeta)
                     .setInstructions(instructions2)
@@ -981,7 +963,7 @@
                     .setHardTimeout(0)
                     .setXid(getNextTransactionId())
                     .build();
-            msglist.add(myMetaEntry);
+            msglist.add(myMetaEntry); */
 
         }
         write(msglist);
@@ -1030,7 +1012,7 @@
             instructions.add(writeInstr);
             instructions.add(gotoInstr);
             OFMessage myIpEntry = factory.buildFlowAdd()
-                    .setTableId(TableId.of(TABLE_IPV4_UNICAST))
+                    .setTableId(TableId.of(TABLE_IPv4_UNICAST))
                     .setMatch(match)
                     .setInstructions(instructions)
                     .setPriority(MAX_PRIORITY) // highest priority for exact
@@ -1046,7 +1028,7 @@
         log.debug("Adding {} next-hop-host-rules in sw {}", msglist.size(), getStringId());
     }
 
-    private static class MplsEntry {
+    private class MplsEntry {
         int labelid;
         int portnum;
 
@@ -1086,15 +1068,18 @@
             OFMatchV3 matchlabel = factory.buildMatchV3()
                     .setOxmList(oxmList).build();
             OFAction poplabel = factory.actions().popMpls(EthType.IPv4);
+            OFAction copyttlin = factory.actions().copyTtlIn();
             OFAction sendTo = null;
             if (lfibEntries.get(i).portnum == OFPort.CONTROLLER.getPortNumber()) {
                 sendTo = factory.actions().output(OFPort.CONTROLLER,
                         OFPCML_NO_BUFFER);
             } else {
+                // after popping send to L3 intf, not MPLS intf
                 sendTo = factory.actions().group(OFGroup.of(
-                        0xa0000000 | lfibEntries.get(i).portnum));
+                        0x20000000 | lfibEntries.get(i).portnum));
             }
             List<OFAction> writeActions = new ArrayList<OFAction>();
+            writeActions.add(copyttlin);
             writeActions.add(poplabel);
             writeActions.add(sendTo);
             OFInstruction writeInstr = factory.instructions().buildWriteActions()