Creating a p4runtime default driver to avoid code duplication between bmv2 and barefoot drivers

Change-Id: Id7f16a284c65278ec1a9ec682da01ddf020343c8
diff --git a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2FlowRuleProgrammable.java b/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2FlowRuleProgrammable.java
deleted file mode 100644
index cd2bf7e..0000000
--- a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2FlowRuleProgrammable.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/*
- * 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.drivers.bmv2;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.onosproject.net.Device;
-import org.onosproject.net.DeviceId;
-import org.onosproject.net.device.DeviceService;
-import org.onosproject.net.driver.AbstractHandlerBehaviour;
-import org.onosproject.net.flow.DefaultFlowEntry;
-import org.onosproject.net.flow.FlowEntry;
-import org.onosproject.net.flow.FlowRule;
-import org.onosproject.net.flow.FlowRuleProgrammable;
-import org.onosproject.net.pi.model.PiPipeconf;
-import org.onosproject.net.pi.model.PiPipelineInterpreter;
-import org.onosproject.net.pi.model.PiPipelineModel;
-import org.onosproject.net.pi.model.PiTableModel;
-import org.onosproject.net.pi.runtime.PiFlowRuleTranslationService;
-import org.onosproject.net.pi.runtime.PiPipeconfService;
-import org.onosproject.net.pi.runtime.PiTableEntry;
-import org.onosproject.net.pi.runtime.PiTableId;
-import org.onosproject.p4runtime.api.P4RuntimeClient;
-import org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType;
-import org.onosproject.p4runtime.api.P4RuntimeController;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static com.google.common.collect.Lists.newArrayList;
-import static org.onosproject.drivers.bmv2.Bmv2FlowRuleProgrammable.Operation.APPLY;
-import static org.onosproject.drivers.bmv2.Bmv2FlowRuleProgrammable.Operation.REMOVE;
-import static org.onosproject.net.flow.FlowEntry.FlowEntryState.ADDED;
-import static org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType.DELETE;
-import static org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType.INSERT;
-
-/**
- * Implementation of the flow rule programmable behaviour for BMv2.
- */
-public class Bmv2FlowRuleProgrammable extends AbstractHandlerBehaviour implements FlowRuleProgrammable {
-
-    private final Logger log = LoggerFactory.getLogger(getClass());
-
-    // Needed to synchronize operations over the same table entry.
-    private static final ConcurrentMap<Bmv2TableEntryReference, Lock> ENTRY_LOCKS = Maps.newConcurrentMap();
-
-    // TODO: replace with distributed store.
-    // Can reuse old BMv2TableEntryService from ONOS 1.6
-    private static final ConcurrentMap<Bmv2TableEntryReference, Bmv2FlowRuleWrapper> ENTRY_STORE =
-            Maps.newConcurrentMap();
-
-    private DeviceId deviceId;
-    private P4RuntimeClient client;
-    private PiPipeconf pipeconf;
-    private PiPipelineModel pipelineModel;
-    private PiPipelineInterpreter interpreter;
-    private PiFlowRuleTranslationService piFlowRuleTranslationService;
-
-    private boolean init() {
-
-        deviceId = handler().data().deviceId();
-
-        P4RuntimeController controller = handler().get(P4RuntimeController.class);
-        if (!controller.hasClient(deviceId)) {
-            log.warn("Unable to find client for {}, aborting flow rule operation", deviceId);
-            return false;
-        }
-
-        PiPipeconfService piPipeconfService = handler().get(PiPipeconfService.class);
-        if (!piPipeconfService.ofDevice(deviceId).isPresent() ||
-                !piPipeconfService.getPipeconf(piPipeconfService.ofDevice(deviceId).get()).isPresent()) {
-            log.warn("Unable to get the pipeconf of {}", deviceId);
-            return false;
-        }
-
-        DeviceService deviceService = handler().get(DeviceService.class);
-        Device device = deviceService.getDevice(deviceId);
-        if (!device.is(PiPipelineInterpreter.class)) {
-            log.warn("Unable to get interpreter of {}", deviceId);
-            return false;
-        }
-
-        client = controller.getClient(deviceId);
-        pipeconf = piPipeconfService.getPipeconf(piPipeconfService.ofDevice(deviceId).get()).get();
-        pipelineModel = pipeconf.pipelineModel();
-        interpreter = device.as(PiPipelineInterpreter.class);
-        piFlowRuleTranslationService = handler().get(PiFlowRuleTranslationService.class);
-
-        return true;
-    }
-
-    @Override
-    public Collection<FlowEntry> getFlowEntries() {
-
-        if (!init()) {
-            return Collections.emptyList();
-        }
-
-        ImmutableList.Builder<FlowEntry> resultBuilder = ImmutableList.builder();
-        List<PiTableEntry> inconsistentEntries = Lists.newArrayList();
-
-        for (PiTableModel tableModel : pipelineModel.tables()) {
-
-            PiTableId piTableId = PiTableId.of(tableModel.name());
-
-            // Only dump tables that are exposed by the interpreter.
-            // The reason is that some P4 targets (e.g. BMv2's simple_switch) use more table than those defined in the
-            // P4 program, to implement other capabilities, e.g. action execution in control flow.
-            if (!interpreter.mapPiTableId(piTableId).isPresent()) {
-                continue; // next table
-            }
-
-            Collection<PiTableEntry> installedEntries;
-            try {
-                installedEntries = client.dumpTable(piTableId, pipeconf).get();
-            } catch (InterruptedException | ExecutionException e) {
-                log.error("Exception while dumping table {} of {}", piTableId, deviceId, e);
-                return Collections.emptyList();
-            }
-
-            for (PiTableEntry installedEntry : installedEntries) {
-
-                Bmv2TableEntryReference entryRef = new Bmv2TableEntryReference(deviceId, piTableId,
-                                                                               installedEntry.matchKey());
-
-                Bmv2FlowRuleWrapper frWrapper = ENTRY_STORE.get(entryRef);
-
-                if (frWrapper == null) {
-                    // Inconsistent entry
-                    inconsistentEntries.add(installedEntry);
-                    continue; // next one.
-                }
-
-                // TODO: implement table entry counter retrieval.
-                long bytes = 0L;
-                long packets = 0L;
-
-                FlowEntry entry = new DefaultFlowEntry(frWrapper.rule(), ADDED, frWrapper.lifeInSeconds(),
-                                                       packets, bytes);
-                resultBuilder.add(entry);
-            }
-        }
-
-        if (inconsistentEntries.size() > 0) {
-            log.warn("Found {} entries in {} that are not known by table entry service," +
-                             " removing them", inconsistentEntries.size(), deviceId);
-            inconsistentEntries.forEach(entry -> log.debug(entry.toString()));
-            // Async remove them.
-            client.writeTableEntries(inconsistentEntries, DELETE, pipeconf);
-        }
-
-        return resultBuilder.build();
-    }
-
-    @Override
-    public Collection<FlowRule> applyFlowRules(Collection<FlowRule> rules) {
-        return processFlowRules(rules, APPLY);
-    }
-
-    @Override
-    public Collection<FlowRule> removeFlowRules(Collection<FlowRule> rules) {
-        return processFlowRules(rules, REMOVE);
-    }
-
-    private Collection<FlowRule> processFlowRules(Collection<FlowRule> rules, Operation operation) {
-
-        if (!init()) {
-            return Collections.emptyList();
-        }
-
-        ImmutableList.Builder<FlowRule> processedFlowRuleListBuilder = ImmutableList.builder();
-
-        // TODO: send write operations in bulk (e.g. all entries to insert, modify or delete).
-        // Instead of calling the client for each one of them.
-
-        for (FlowRule rule : rules) {
-
-            PiTableEntry piTableEntry;
-
-            try {
-                piTableEntry = piFlowRuleTranslationService.translate(rule, pipeconf);
-            } catch (PiFlowRuleTranslationService.PiFlowRuleTranslationException e) {
-                log.warn("Unable to translate flow rule: {} - {}", e.getMessage(), rule);
-                continue; // next rule
-            }
-
-            PiTableId tableId = piTableEntry.table();
-            Bmv2TableEntryReference entryRef = new Bmv2TableEntryReference(deviceId, tableId, piTableEntry.matchKey());
-
-            Lock lock = ENTRY_LOCKS.computeIfAbsent(entryRef, k -> new ReentrantLock());
-            lock.lock();
-
-            try {
-
-                Bmv2FlowRuleWrapper frWrapper = ENTRY_STORE.get(entryRef);
-
-                WriteOperationType opType;
-                if (operation == Operation.APPLY) {
-                    opType = INSERT;
-                    if (frWrapper != null) {
-                        // We've seen some strange error when trying to modify existing flow rules.
-                        // Remove before re-adding the modified one.
-                        try {
-                            if (client.writeTableEntries(newArrayList(piTableEntry), DELETE, pipeconf).get()) {
-                                frWrapper = null;
-                            } else {
-                                log.warn("Unable to DELETE table entry (before re-adding) in {}: {}",
-                                         deviceId, piTableEntry);
-                            }
-                        } catch (InterruptedException | ExecutionException e) {
-                            log.warn("Exception while deleting table entry:", operation.name(), e);
-                        }
-                    }
-                } else {
-                    opType = DELETE;
-                }
-
-                try {
-                    if (client.writeTableEntries(newArrayList(piTableEntry), opType, pipeconf).get()) {
-                        processedFlowRuleListBuilder.add(rule);
-                        frWrapper = new Bmv2FlowRuleWrapper(rule, System.currentTimeMillis());
-                    } else {
-                        log.warn("Unable to {} table entry in {}: {}", opType.name(), deviceId, piTableEntry);
-                    }
-                } catch (InterruptedException | ExecutionException e) {
-                    log.warn("Exception while performing {} table entry operation:", operation.name(), e);
-                }
-
-                // Update entryRef binding in table entry service.
-                if (frWrapper != null) {
-                    ENTRY_STORE.put(entryRef, frWrapper);
-                } else {
-                    ENTRY_STORE.remove(entryRef);
-                }
-
-            } finally {
-                lock.unlock();
-            }
-        }
-
-        return processedFlowRuleListBuilder.build();
-    }
-
-    enum Operation {
-        APPLY, REMOVE
-    }
-}
\ No newline at end of file
diff --git a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2FlowRuleWrapper.java b/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2FlowRuleWrapper.java
deleted file mode 100644
index 879783b..0000000
--- a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2FlowRuleWrapper.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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.drivers.bmv2;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import org.onosproject.net.flow.FlowRule;
-
-/**
- * A wrapper for a ONOS flow rule installed on a BMv2 device.
- */
-@Beta
-final class Bmv2FlowRuleWrapper {
-
-    private final FlowRule rule;
-    private final long installedOnMillis;
-
-    /**
-     * Creates a new flow rule wrapper.
-     *
-     * @param rule              a flow rule
-     * @param installedOnMillis the time (in milliseconds, since January 1, 1970 UTC) when the flow rule was installed
-     *                          on the device
-     */
-    Bmv2FlowRuleWrapper(FlowRule rule, long installedOnMillis) {
-        this.rule = rule;
-        this.installedOnMillis = installedOnMillis;
-    }
-
-    /**
-     * Returns the flow rule contained by this wrapper.
-     *
-     * @return a flow rule
-     */
-    FlowRule rule() {
-        return rule;
-    }
-
-    /**
-     * Return the number of seconds since when this flow rule was installed on the device.
-     *
-     * @return an integer value
-     */
-    long lifeInSeconds() {
-        return (System.currentTimeMillis() - installedOnMillis) / 1000;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(rule, installedOnMillis);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null || getClass() != obj.getClass()) {
-            return false;
-        }
-        final Bmv2FlowRuleWrapper other = (Bmv2FlowRuleWrapper) obj;
-        return Objects.equal(this.rule, other.rule)
-                && Objects.equal(this.installedOnMillis, other.installedOnMillis);
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(this)
-                .add("rule", rule)
-                .add("installedOnMillis", installedOnMillis)
-                .toString();
-    }
-}
diff --git a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2GroupProgrammable.java b/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2GroupProgrammable.java
deleted file mode 100644
index 954ecf8..0000000
--- a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2GroupProgrammable.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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.drivers.bmv2;
-
-import org.onosproject.net.Device;
-import org.onosproject.net.DeviceId;
-import org.onosproject.net.driver.AbstractHandlerBehaviour;
-import org.onosproject.net.group.GroupBucket;
-import org.onosproject.net.group.GroupDescription;
-import org.onosproject.net.group.GroupOperation;
-import org.onosproject.net.group.GroupOperations;
-import org.onosproject.net.group.GroupProgrammable;
-import org.onosproject.net.pi.model.PiPipelineInterpreter;
-import org.onosproject.net.pi.runtime.PiAction;
-import org.onosproject.net.pi.runtime.PiActionGroup;
-import org.onosproject.net.pi.runtime.PiActionGroupId;
-import org.onosproject.net.pi.runtime.PiActionGroupMember;
-import org.onosproject.net.pi.runtime.PiActionGroupMemberId;
-import org.onosproject.net.pi.runtime.PiTableId;
-
-import java.nio.ByteBuffer;
-
-public class Bmv2GroupProgrammable extends AbstractHandlerBehaviour implements GroupProgrammable {
-
-    /*
-    Work in progress.
-     */
-
-    private Device device;
-
-    /*
-    About action groups in P4runtime:
-    The type field is a place holder in p4runtime.proto right now, and we haven't defined it yet. You can assume all
-    the groups are "select" as per the OF spec. As a remainder, in the P4 terminology a member corresponds to an OF
-    bucket. Each member can also be used directly in the match table (kind of like an OF indirect group).
-     */
-
-    @Override
-    public void performGroupOperation(DeviceId deviceId, GroupOperations groupOps) {
-
-        for (GroupOperation groupOp : groupOps.operations()) {
-            switch (groupOp.opType()) {
-                case ADD:
-                    addGroup(deviceId, groupOp);
-                    break;
-                default:
-                    throw new UnsupportedOperationException();
-            }
-        }
-    }
-
-    private void addGroup(DeviceId deviceId, GroupOperation groupOp) {
-
-        // Most of this logic can go in a core service, e.g. PiGroupTranslationService
-
-        // From a P4Runtime perspective, we need first to insert members, then the group.
-
-        PiActionGroupId piActionGroupId = PiActionGroupId.of(groupOp.groupId().id());
-
-        PiActionGroup.Builder piActionGroupBuilder = PiActionGroup.builder()
-                .withId(piActionGroupId)
-                .withType(PiActionGroup.Type.SELECT);
-
-        if (groupOp.groupType() != GroupDescription.Type.SELECT) {
-            // log error
-        }
-
-        int bucketIdx = 0;
-        for (GroupBucket bucket : groupOp.buckets().buckets()) {
-            /*
-            Problem:
-            In P4Runtime action group members, i.e. action buckets, are associated to a numeric ID chosen
-            at member insertion time. This ID must be unique for the whole action profile (i.e. the group table in
-            OpenFlow). In ONOS, GroupBucket doesn't specify any ID.
-
-            Solutions:
-            - Change GroupBucket API to force application wanting to perform group operations to specify a member id.
-            - Maintain state to dynamically allocate/deallocate member IDs, e.g. in a dedicated service, or in a
-            P4Runtime Group Provider.
-
-            Hack:
-            Statically derive member ID by combining groupId and position of the bucket in the list.
-             */
-            int memberId = ByteBuffer.allocate(4)
-                    .putShort((short) (piActionGroupId.id() % 2 ^ 16))
-                    .putShort((short) (bucketIdx % 2 ^ 16))
-                    .getInt();
-
-            // Need an interpreter to map the bucket treatment to a PI action
-
-            if (!device.is(PiPipelineInterpreter.class)) {
-                // log error
-            }
-
-            PiPipelineInterpreter interpreter = device.as(PiPipelineInterpreter.class);
-
-            /*
-            Problem:
-            In P4Runtime, action profiles (i.e. group tables) are specific to one or more tables.
-            Mapping of treatments depends on the target table. How do we derive the target table from here?
-
-            Solution:
-            - Change GroupDescription to allow applications to specify a table where this group will be called from.
-
-            Hack:
-            Assume we support pipelines with only one action profile associated to only one table, i.e. derive the
-            table ID by looking at the P4Info.
-             */
-
-            PiTableId piTableId = PiTableId.of("derive from P4Info");
-
-
-            PiAction action = null;
-            try {
-                action = interpreter.mapTreatment(bucket.treatment(), piTableId);
-            } catch (PiPipelineInterpreter.PiInterpreterException e) {
-                // log error
-            }
-
-            PiActionGroupMember member = PiActionGroupMember.builder()
-                    .withId(PiActionGroupMemberId.of(memberId))
-                    .withAction(action)
-                    .withWeight(bucket.weight())
-                    .build();
-
-            piActionGroupBuilder.addMember(member);
-
-            // Use P4RuntimeClient to install member;
-            // TODO: implement P4RuntimeClient method.
-        }
-
-        PiActionGroup piActionGroup = piActionGroupBuilder.build();
-
-        // Use P4RuntimeClient to insert group.
-        // TODO: implement P4RuntimeClient method.
-    }
-}
diff --git a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2Handshaker.java b/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2Handshaker.java
deleted file mode 100644
index 808561d..0000000
--- a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2Handshaker.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.drivers.bmv2;
-
-import io.grpc.ManagedChannelBuilder;
-import io.grpc.netty.NettyChannelBuilder;
-import org.onosproject.net.DeviceId;
-import org.onosproject.net.MastershipRole;
-import org.onosproject.net.device.DeviceHandshaker;
-import org.onosproject.net.driver.AbstractHandlerBehaviour;
-import org.onosproject.net.driver.DriverData;
-import org.onosproject.p4runtime.api.P4RuntimeController;
-import org.slf4j.Logger;
-
-import java.util.concurrent.CompletableFuture;
-
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Implementation of DeviceHandshaker for BMv2.
- */
-public class Bmv2Handshaker extends AbstractHandlerBehaviour
-        implements DeviceHandshaker {
-
-    private final Logger log = getLogger(getClass());
-
-    // TODO: consider abstract class with empty connect method and  implementation into a protected one for reusability.
-
-    @Override
-    public CompletableFuture<Boolean> connect() {
-        return CompletableFuture.supplyAsync(this::doConnect);
-    }
-
-    private boolean doConnect() {
-
-        P4RuntimeController controller = handler().get(P4RuntimeController.class);
-
-        DeviceId deviceId = handler().data().deviceId();
-        // DeviceKeyService deviceKeyService = handler().get(DeviceKeyService.class);
-        DriverData data = data();
-
-        String serverAddr = data.value("p4runtime_ip");
-        int serverPort = Integer.valueOf(data.value("p4runtime_port"));
-        int p4DeviceId = Integer.valueOf(data.value("p4runtime_deviceId"));
-
-        ManagedChannelBuilder channelBuilder = NettyChannelBuilder
-                .forAddress(serverAddr, serverPort)
-                .usePlaintext(true);
-
-        if (!controller.createClient(deviceId, p4DeviceId, channelBuilder)) {
-            log.warn("Unable to create P4runtime client for {}", deviceId);
-            return false;
-        }
-
-        // TODO: gNMI handling
-
-        return true;
-    }
-
-    @Override
-    public CompletableFuture<Boolean> disconnect() {
-        return CompletableFuture.supplyAsync(() -> {
-            P4RuntimeController controller = handler().get(P4RuntimeController.class);
-            DeviceId deviceId = handler().data().deviceId();
-            controller.removeClient(deviceId);
-            return true;
-        });
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isReachable() {
-        return CompletableFuture.supplyAsync(() -> {
-            P4RuntimeController controller = handler().get(P4RuntimeController.class);
-            DeviceId deviceId = handler().data().deviceId();
-            return controller.isReacheable(deviceId);
-        });
-    }
-
-    @Override
-    public CompletableFuture<MastershipRole> roleChanged(MastershipRole newRole) {
-        CompletableFuture<MastershipRole> result = new CompletableFuture<>();
-        log.warn("roleChanged not implemented");
-        result.complete(MastershipRole.MASTER);
-        // TODO.
-        return result;
-    }
-}
diff --git a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2PacketProgrammable.java b/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2PacketProgrammable.java
deleted file mode 100644
index 50dfe1d..0000000
--- a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2PacketProgrammable.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.drivers.bmv2;
-
-import org.onosproject.net.Device;
-import org.onosproject.net.DeviceId;
-import org.onosproject.net.device.DeviceService;
-import org.onosproject.net.driver.AbstractHandlerBehaviour;
-import org.onosproject.net.packet.OutboundPacket;
-import org.onosproject.net.packet.PacketProgrammable;
-import org.onosproject.net.pi.model.PiPipeconf;
-import org.onosproject.net.pi.model.PiPipelineInterpreter;
-import org.onosproject.net.pi.runtime.PiPacketOperation;
-import org.onosproject.net.pi.runtime.PiPipeconfService;
-import org.onosproject.p4runtime.api.P4RuntimeClient;
-import org.onosproject.p4runtime.api.P4RuntimeController;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-
-/**
- * Packet Programmable behaviour for BMv2 devices.
- */
-public class Bmv2PacketProgrammable extends AbstractHandlerBehaviour implements PacketProgrammable {
-    private final Logger log = LoggerFactory.getLogger(getClass());
-
-    @Override
-    public void emit(OutboundPacket packet) {
-
-        DeviceId deviceId = handler().data().deviceId();
-        P4RuntimeController controller = handler().get(P4RuntimeController.class);
-        if (!controller.hasClient(deviceId)) {
-            log.warn("Unable to find client for {}, aborting the sending packet", deviceId);
-            return;
-        }
-
-        P4RuntimeClient client = controller.getClient(deviceId);
-        PiPipeconfService piPipeconfService = handler().get(PiPipeconfService.class);
-
-        final PiPipeconf pipeconf;
-        if (piPipeconfService.ofDevice(deviceId).isPresent() &&
-                piPipeconfService.getPipeconf(piPipeconfService.ofDevice(deviceId).get()).isPresent()) {
-            pipeconf = piPipeconfService.getPipeconf(piPipeconfService.ofDevice(deviceId).get()).get();
-        } else {
-            log.warn("Unable to get the pipeconf of {}", deviceId);
-            return;
-        }
-
-        DeviceService deviceService = handler().get(DeviceService.class);
-        Device device = deviceService.getDevice(deviceId);
-        final PiPipelineInterpreter interpreter = device.is(PiPipelineInterpreter.class)
-                ? device.as(PiPipelineInterpreter.class) : null;
-        if (!device.is(PiPipelineInterpreter.class)) {
-            log.warn("Device {} unable to instantiate interpreter of pipeconf {}", deviceId, pipeconf.id());
-            return;
-        }
-
-        try {
-            Collection<PiPacketOperation> operations = interpreter.mapOutboundPacket(packet);
-            operations.forEach(piPacketOperation -> {
-                log.debug("Doing PiPacketOperation {}", piPacketOperation);
-                client.packetOut(piPacketOperation, pipeconf);
-            });
-        } catch (PiPipelineInterpreter.PiInterpreterException e) {
-            log.error("Interpreter of pipeconf {} was unable to translate outbound packet: {}",
-                    pipeconf.id(), e.getMessage());
-        }
-    }
-}
diff --git a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2TableEntryReference.java b/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2TableEntryReference.java
deleted file mode 100644
index 9b99faa..0000000
--- a/drivers/bmv2/src/main/java/org/onosproject/drivers/bmv2/Bmv2TableEntryReference.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.drivers.bmv2;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import org.onosproject.net.DeviceId;
-import org.onosproject.net.pi.runtime.PiMatchKey;
-import org.onosproject.net.pi.runtime.PiTableId;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-public final class Bmv2TableEntryReference {
-
-    private final DeviceId deviceId;
-    private final PiTableId tableId;
-    private final PiMatchKey matchKey;
-
-    /**
-     * Creates a new table entry reference.
-     *
-     * @param deviceId a device ID
-     * @param tableId  a table name
-     * @param matchKey a match key
-     */
-    public Bmv2TableEntryReference(DeviceId deviceId, PiTableId tableId, PiMatchKey matchKey) {
-        this.deviceId = checkNotNull(deviceId);
-        this.tableId = checkNotNull(tableId);
-        this.matchKey = checkNotNull(matchKey);
-    }
-
-    /**
-     * Returns the device ID of this table entry reference.
-     *
-     * @return a device ID
-     */
-    public DeviceId deviceId() {
-        return deviceId;
-    }
-
-    /**
-     * Returns the table id of this table entry reference.
-     *
-     * @return a table name
-     */
-    public PiTableId tableId() {
-        return tableId;
-    }
-
-    /**
-     * Returns the match key of this table entry reference.
-     *
-     * @return a match key
-     */
-    public PiMatchKey matchKey() {
-        return matchKey;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(deviceId, tableId, matchKey);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null || getClass() != obj.getClass()) {
-            return false;
-        }
-        final Bmv2TableEntryReference other = (Bmv2TableEntryReference) obj;
-        return Objects.equal(this.deviceId, other.deviceId)
-                && Objects.equal(this.tableId, other.tableId)
-                && Objects.equal(this.matchKey, other.matchKey);
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(this)
-                .add("deviceId", deviceId)
-                .add("tableId", tableId)
-                .add("matchKey", matchKey)
-                .toString();
-    }
-}
diff --git a/drivers/bmv2/src/main/resources/bmv2-drivers.xml b/drivers/bmv2/src/main/resources/bmv2-drivers.xml
index af73d95..97972b3 100644
--- a/drivers/bmv2/src/main/resources/bmv2-drivers.xml
+++ b/drivers/bmv2/src/main/resources/bmv2-drivers.xml
@@ -15,13 +15,7 @@
   ~ limitations under the License.
   -->
 <drivers>
-    <driver name="bmv2" manufacturer="p4.org" hwVersion="master" swVersion="master">
-        <behaviour api="org.onosproject.net.device.DeviceHandshaker"
-                   impl="org.onosproject.drivers.bmv2.Bmv2Handshaker"/>
-        <behaviour api="org.onosproject.net.packet.PacketProgrammable"
-                   impl="org.onosproject.drivers.bmv2.Bmv2PacketProgrammable"/>
-        <behaviour api="org.onosproject.net.flow.FlowRuleProgrammable"
-                   impl="org.onosproject.drivers.bmv2.Bmv2FlowRuleProgrammable"/>
+    <driver name="bmv2" manufacturer="p4.org" hwVersion="master" swVersion="master" extends="p4runtime">
         <behaviour api="org.onosproject.net.pi.model.PiPipelineProgrammable"
                    impl="org.onosproject.drivers.bmv2.Bmv2PipelineProgrammable"/>
     </driver>