Creating a p4runtime default driver to avoid code duplication between bmv2 and barefoot drivers
Change-Id: Id7f16a284c65278ec1a9ec682da01ddf020343c8
diff --git a/drivers/bmv2/BUCK b/drivers/bmv2/BUCK
index a8dfd58..30ee2ab 100644
--- a/drivers/bmv2/BUCK
+++ b/drivers/bmv2/BUCK
@@ -31,5 +31,6 @@
'org.onosproject.generaldeviceprovider',
'org.onosproject.protocols.p4runtime',
'org.onosproject.p4runtime',
+ 'org.onosproject.drivers.p4runtime'
],
)
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>