ONOS-7050 Refactored P4Runtime GP to use distributed stores
Change-Id: I155bd0d848bfad2a198d220fcb35ab2ad2daa56a
diff --git a/core/api/src/main/java/org/onosproject/net/pi/runtime/PiActionGroupHandle.java b/core/api/src/main/java/org/onosproject/net/pi/runtime/PiActionGroupHandle.java
index 6c464de..25a035b 100644
--- a/core/api/src/main/java/org/onosproject/net/pi/runtime/PiActionGroupHandle.java
+++ b/core/api/src/main/java/org/onosproject/net/pi/runtime/PiActionGroupHandle.java
@@ -28,9 +28,20 @@
@Beta
public final class PiActionGroupHandle extends PiHandle<PiActionGroup> {
+ private PiActionGroupHandle(DeviceId deviceId, PiActionGroup group) {
+ super(deviceId, group);
+ }
- private PiActionGroupHandle(DeviceId deviceId, PiActionGroup actionGroup) {
- super(deviceId, actionGroup);
+ /**
+ * Creates a new handle for the given device ID and PI action group.
+ *
+ * @param deviceId device ID
+ * @param group PI action group
+ * @return PI action group handle
+ */
+ public static PiActionGroupHandle of(DeviceId deviceId,
+ PiActionGroup group) {
+ return new PiActionGroupHandle(deviceId, group);
}
@Override
diff --git a/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/P4RuntimeGroupProgrammable.java b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/P4RuntimeGroupProgrammable.java
index 3cba2d5..fb5e892 100644
--- a/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/P4RuntimeGroupProgrammable.java
+++ b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/P4RuntimeGroupProgrammable.java
@@ -17,180 +17,86 @@
package org.onosproject.drivers.p4runtime;
import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.onosproject.core.GroupId;
-import org.onosproject.net.Device;
+import org.onosproject.drivers.p4runtime.mirror.P4RuntimeGroupMirror;
import org.onosproject.net.DeviceId;
-import org.onosproject.net.device.DeviceService;
+import org.onosproject.net.group.DefaultGroup;
import org.onosproject.net.group.Group;
import org.onosproject.net.group.GroupOperation;
import org.onosproject.net.group.GroupOperations;
import org.onosproject.net.group.GroupProgrammable;
import org.onosproject.net.group.GroupStore;
import org.onosproject.net.pi.model.PiActionProfileId;
+import org.onosproject.net.pi.model.PiActionProfileModel;
import org.onosproject.net.pi.runtime.PiActionGroup;
-import org.onosproject.net.pi.runtime.PiActionGroupId;
+import org.onosproject.net.pi.runtime.PiActionGroupHandle;
+import org.onosproject.net.pi.service.PiGroupTranslator;
+import org.onosproject.net.pi.service.PiTranslatedEntity;
import org.onosproject.net.pi.service.PiTranslationException;
-import org.onosproject.p4runtime.api.P4RuntimeClient;
-import org.onosproject.p4runtime.api.P4RuntimeGroupReference;
-import org.onosproject.p4runtime.api.P4RuntimeGroupWrapper;
import org.slf4j.Logger;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
+import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import static org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType.DELETE;
+import static org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType.INSERT;
import static org.slf4j.LoggerFactory.getLogger;
/**
* Implementation of the group programmable behaviour for P4Runtime.
*/
-public class P4RuntimeGroupProgrammable extends AbstractP4RuntimeHandlerBehaviour implements GroupProgrammable {
- private static final String ACT_GRP_MEMS = "action group members";
- private static final String DELETE = "delete";
- private static final String ACT_GRP = "action group";
- private static final String INSERT = "insert";
+public class P4RuntimeGroupProgrammable
+ extends AbstractP4RuntimeHandlerBehaviour
+ implements GroupProgrammable {
+
+ private enum Operation {
+ APPLY, REMOVE
+ }
+
+ private static final String ACT_GRP_MEMS_STR = "action group members";
+ private static final String DELETE_STR = "delete";
+ private static final String ACT_GRP_STR = "action group";
+ private static final String INSERT_STR = "insert";
+
private static final Logger log = getLogger(P4RuntimeGroupProgrammable.class);
- /*
- * 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).
- */
+ // If true, we ignore re-installing groups that are already known in the
+ // device mirror.
+ private boolean checkMirrorBeforeUpdate = true;
- // TODO: make this attribute configurable by child drivers (e.g. BMv2 or Tofino)
- /*
- When updating an existing rule, if true, we issue a DELETE operation before inserting the new one, otherwise we
- issue a MODIFY operation. This is useful fore devices that do not support MODIFY operations for table entries.
- */
- private boolean deleteBeforeUpdate = true;
-
- // TODO: can remove this check as soon as the multi-apply-per-same-flow rule bug is fixed.
- /*
- If true, we ignore re-installing rules that are already known in the ENTRY_STORE, i.e. same match key and action.
- */
- private boolean checkStoreBeforeUpdate = true;
+ private GroupStore groupStore;
+ private P4RuntimeGroupMirror groupMirror;
+ private PiGroupTranslator translator;
// Needed to synchronize operations over the same group.
- private static final Map<P4RuntimeGroupReference, Lock> GROUP_LOCKS = Maps.newConcurrentMap();
-
- // TODO: replace with distribute store
- private static final Map<P4RuntimeGroupReference, P4RuntimeGroupWrapper> GROUP_STORE = Maps.newConcurrentMap();
+ private static final Map<PiActionGroupHandle, Lock> GROUP_LOCKS =
+ Maps.newConcurrentMap();
@Override
- public void performGroupOperation(DeviceId deviceId, GroupOperations groupOps) {
+ protected boolean setupBehaviour() {
+ if (!super.setupBehaviour()) {
+ return false;
+ }
+ groupMirror = this.handler().get(P4RuntimeGroupMirror.class);
+ groupStore = handler().get(GroupStore.class);
+ translator = piTranslationService.groupTranslator();
+ return true;
+ }
+
+ @Override
+ public void performGroupOperation(DeviceId deviceId,
+ GroupOperations groupOps) {
if (!setupBehaviour()) {
return;
}
-
- Device device = handler().get(DeviceService.class).getDevice(deviceId);
-
- for (GroupOperation groupOp : groupOps.operations()) {
- processGroupOp(device, groupOp);
- }
- }
-
- private void processGroupOp(Device device, GroupOperation groupOp) {
- GroupId groupId = groupOp.groupId();
- GroupStore groupStore = handler().get(GroupStore.class);
- Group group = groupStore.getGroup(device.id(), groupId);
-
- PiActionGroup piActionGroup;
- try {
- piActionGroup = piTranslationService.groupTranslator().translate(group, pipeconf);
- } catch (PiTranslationException e) {
- log.warn("Unable translate group, aborting group operation {}: {}", groupOp.opType(), e.getMessage());
- return;
- }
-
- P4RuntimeGroupReference groupRef = new P4RuntimeGroupReference(deviceId, piActionGroup.actionProfileId(),
- piActionGroup.id());
-
- Lock lock = GROUP_LOCKS.computeIfAbsent(groupRef, k -> new ReentrantLock());
- lock.lock();
-
- try {
- P4RuntimeGroupWrapper oldGroupWrapper = GROUP_STORE.get(groupRef);
- P4RuntimeGroupWrapper newGroupWrapper = new P4RuntimeGroupWrapper(piActionGroup, group,
- System.currentTimeMillis());
- switch (groupOp.opType()) {
- case ADD:
- case MODIFY:
- if (writeGroupToDevice(oldGroupWrapper, piActionGroup)) {
- GROUP_STORE.put(groupRef, newGroupWrapper);
- }
- break;
- case DELETE:
- if (deleteGroupFromDevice(piActionGroup)) {
- GROUP_STORE.remove(groupRef);
- }
- break;
- default:
- log.warn("Group operation {} not supported", groupOp.opType());
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Installs action group and members to device via client interface.
- *
- * @param oldGroupWrapper old group wrapper for the group; null if not exists
- * @param piActionGroup the action group to be installed
- * @return true if install success; false otherwise
- */
- private boolean writeGroupToDevice(P4RuntimeGroupWrapper oldGroupWrapper, PiActionGroup piActionGroup) {
- boolean success = true;
- CompletableFuture<Boolean> writeSuccess;
- if (checkStoreBeforeUpdate && oldGroupWrapper != null &&
- oldGroupWrapper.piActionGroup().equals(piActionGroup)) {
- // Action group already exists, ignore it
- return true;
- }
- if (deleteBeforeUpdate && oldGroupWrapper != null) {
- success = deleteGroupFromDevice(oldGroupWrapper.piActionGroup());
- }
- writeSuccess = client.writeActionGroupMembers(piActionGroup,
- P4RuntimeClient.WriteOperationType.INSERT,
- pipeconf);
- success = success && completeSuccess(writeSuccess, ACT_GRP_MEMS, INSERT);
-
- writeSuccess = client.writeActionGroup(piActionGroup,
- P4RuntimeClient.WriteOperationType.INSERT,
- pipeconf);
- success = success && completeSuccess(writeSuccess, ACT_GRP, INSERT);
- return success;
- }
-
- private boolean deleteGroupFromDevice(PiActionGroup piActionGroup) {
- boolean success;
- CompletableFuture<Boolean> writeSuccess;
- writeSuccess = client.writeActionGroup(piActionGroup,
- P4RuntimeClient.WriteOperationType.DELETE,
- pipeconf);
- success = completeSuccess(writeSuccess, ACT_GRP, DELETE);
- writeSuccess = client.writeActionGroupMembers(piActionGroup,
- P4RuntimeClient.WriteOperationType.DELETE,
- pipeconf);
- success = success && completeSuccess(writeSuccess, ACT_GRP_MEMS, DELETE);
- return success;
- }
-
- private boolean completeSuccess(CompletableFuture<Boolean> completableFuture,
- String topic, String action) {
- try {
- return completableFuture.get();
- } catch (InterruptedException | ExecutionException e) {
- log.warn("Can't {} {} due to {}", action, topic, e.getMessage());
- return false;
- }
+ groupOps.operations().forEach(op -> processGroupOp(deviceId, op));
}
@Override
@@ -198,58 +104,147 @@
if (!setupBehaviour()) {
return Collections.emptyList();
}
+ return pipeconf.pipelineModel().actionProfiles().stream()
+ .map(PiActionProfileModel::id)
+ .flatMap(this::streamGroupsFromDevice)
+ .collect(Collectors.toList());
+ }
- Collection<Group> result = Sets.newHashSet();
- Collection<PiActionProfileId> piActionProfileIds = Sets.newHashSet();
+ private void processGroupOp(DeviceId deviceId, GroupOperation groupOp) {
+ final Group pdGroup = groupStore.getGroup(deviceId, groupOp.groupId());
- // TODO: find better way to get all action profile ids. e.g. by providing them in the interpreter
- GROUP_STORE.forEach((groupRef, wrapper) -> piActionProfileIds.add(groupRef.actionProfileId()));
+ final PiActionGroup piGroup;
+ try {
+ piGroup = translator.translate(pdGroup, pipeconf);
+ } catch (PiTranslationException e) {
+ log.warn("Unable translate group, aborting {} operation: {}",
+ groupOp.opType(), e.getMessage());
+ return;
+ }
- AtomicBoolean success = new AtomicBoolean(true);
- piActionProfileIds.forEach(actionProfileId -> {
- Collection<PiActionGroup> piActionGroups = Sets.newHashSet();
- try {
- Collection<PiActionGroup> groupsFromDevice =
- client.dumpGroups(actionProfileId, pipeconf).get();
- if (groupsFromDevice == null) {
- // Got error
- success.set(false);
- } else {
- piActionGroups.addAll(groupsFromDevice);
- }
- } catch (ExecutionException | InterruptedException e) {
- log.error("Exception while dumping groups for action profile {}: {}",
- actionProfileId.id(), deviceId, e);
- success.set(false);
+ final PiActionGroupHandle handle = PiActionGroupHandle.of(deviceId, piGroup);
+
+ final PiActionGroup groupOnDevice = groupMirror.get(handle) == null
+ ? null
+ : groupMirror.get(handle).entry();
+
+ final Lock lock = GROUP_LOCKS.computeIfAbsent(handle, k -> new ReentrantLock());
+ lock.lock();
+ try {
+ final Operation operation;
+ switch (groupOp.opType()) {
+ case ADD:
+ case MODIFY:
+ operation = Operation.APPLY;
+ break;
+ case DELETE:
+ operation = Operation.REMOVE;
+ break;
+ default:
+ log.warn("Group operation {} not supported", groupOp.opType());
+ return;
}
-
- piActionGroups.forEach(piActionGroup -> {
- PiActionGroupId actionGroupId = piActionGroup.id();
- P4RuntimeGroupReference groupRef =
- new P4RuntimeGroupReference(deviceId, actionProfileId, actionGroupId);
- P4RuntimeGroupWrapper wrapper = GROUP_STORE.get(groupRef);
-
- if (wrapper == null) {
- // group exists in client, but can't find in ONOS
- log.warn("Can't find action profile group {} from local store.",
- groupRef);
- return;
- }
- if (!wrapper.piActionGroup().equals(piActionGroup)) {
- log.warn("Group from device is different to group from local store.");
- return;
- }
- result.add(wrapper.group());
-
- });
- });
-
- if (!success.get()) {
- // Got error while dump groups from device.
- return Collections.emptySet();
- } else {
- return result;
+ processPiGroup(handle, piGroup,
+ groupOnDevice, pdGroup, operation);
+ } finally {
+ lock.unlock();
}
}
+ private void processPiGroup(PiActionGroupHandle handle,
+ PiActionGroup groupToApply,
+ PiActionGroup groupOnDevice,
+ Group pdGroup, Operation operation) {
+ if (operation == Operation.APPLY) {
+ if (groupOnDevice != null) {
+ if (checkMirrorBeforeUpdate
+ && groupOnDevice.equals(groupToApply)) {
+ // Group on device has the same members, ignore operation.
+ return;
+ }
+ // Remove before adding it.
+ processPiGroup(handle, groupToApply, groupOnDevice,
+ pdGroup, Operation.REMOVE);
+ }
+ if (writeGroupToDevice(groupToApply)) {
+ groupMirror.put(handle, groupToApply);
+ translator.learn(handle, new PiTranslatedEntity<>(
+ pdGroup, groupToApply, handle));
+ }
+ } else {
+ if (deleteGroupFromDevice(groupToApply)) {
+ groupMirror.remove(handle);
+ translator.forget(handle);
+ }
+ }
+ }
+
+ private boolean writeGroupToDevice(PiActionGroup groupToApply) {
+ // First insert members, then group.
+ // The operation is deemed successful if both operations are successful.
+ // FIXME: add transactional semantics, i.e. remove members if group fails.
+ final boolean membersSuccess = completeFuture(
+ client.writeActionGroupMembers(groupToApply, INSERT, pipeconf),
+ ACT_GRP_MEMS_STR, INSERT_STR);
+ return membersSuccess && completeFuture(
+ client.writeActionGroup(groupToApply, INSERT, pipeconf),
+ ACT_GRP_STR, INSERT_STR);
+ }
+
+ private boolean deleteGroupFromDevice(PiActionGroup piActionGroup) {
+ // First delete group, then members.
+ // The operation is deemed successful if both operations are successful.
+ final boolean groupSuccess = completeFuture(
+ client.writeActionGroup(piActionGroup, DELETE, pipeconf),
+ ACT_GRP_STR, DELETE_STR);
+ return groupSuccess && completeFuture(
+ client.writeActionGroupMembers(piActionGroup, DELETE, pipeconf),
+ ACT_GRP_MEMS_STR, DELETE_STR);
+ }
+
+ private boolean completeFuture(CompletableFuture<Boolean> completableFuture,
+ String topic, String action) {
+ try {
+ if (completableFuture.get()) {
+ return true;
+ } else {
+ log.warn("Unable to {} {}", action, topic);
+ return false;
+ }
+ } catch (InterruptedException | ExecutionException e) {
+ log.warn("Exception while performing {} {}: {}", action, topic, e.getMessage());
+ log.debug("Exception", e);
+ return false;
+ }
+ }
+
+ private Stream<Group> streamGroupsFromDevice(PiActionProfileId actProfId) {
+ try {
+ // Read PI groups and return original PD one.
+ return client.dumpGroups(actProfId, pipeconf).get().stream()
+ .map(this::forgeGroupEntry)
+ .filter(Objects::nonNull);
+ } catch (ExecutionException | InterruptedException e) {
+ log.error("Exception while dumping groups from action profile '{}' on {}: {}",
+ actProfId.id(), deviceId, e);
+ return Stream.empty();
+ }
+ }
+
+ private Group forgeGroupEntry(PiActionGroup piGroup) {
+ final PiActionGroupHandle handle = PiActionGroupHandle.of(deviceId, piGroup);
+ if (!translator.lookup(handle).isPresent()) {
+ log.warn("Missing PI group from translation store: {} - {}:{}",
+ pipeconf.id(), piGroup.actionProfileId(),
+ piGroup.id());
+ return null;
+ }
+ final long life = groupMirror.get(handle) != null
+ ? groupMirror.get(handle).lifeSec() : 0;
+ final Group original = translator.lookup(handle).get().original();
+ final DefaultGroup forgedGroup = new DefaultGroup(original.id(), original);
+ forgedGroup.setState(Group.GroupState.ADDED);
+ forgedGroup.setLife(life);
+ return forgedGroup;
+ }
}
diff --git a/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/mirror/DistributedP4RuntimeGroupMirror.java b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/mirror/DistributedP4RuntimeGroupMirror.java
new file mode 100644
index 0000000..4c963a6
--- /dev/null
+++ b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/mirror/DistributedP4RuntimeGroupMirror.java
@@ -0,0 +1,50 @@
+/*
+ * 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.p4runtime.mirror;
+
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Service;
+import org.onlab.util.KryoNamespace;
+import org.onosproject.net.pi.runtime.PiActionGroup;
+import org.onosproject.net.pi.runtime.PiActionGroupHandle;
+import org.onosproject.store.serializers.KryoNamespaces;
+
+/**
+ * Distributed implementation of a P4Runtime group mirror.
+ */
+@Component(immediate = true)
+@Service
+public final class DistributedP4RuntimeGroupMirror
+ extends AbstractDistributedP4RuntimeMirror
+ <PiActionGroupHandle, PiActionGroup>
+ implements P4RuntimeGroupMirror {
+
+ private static final String DIST_MAP_NAME = "onos-p4runtime-group-mirror";
+
+ @Override
+ String mapName() {
+ return DIST_MAP_NAME;
+ }
+
+ @Override
+ KryoNamespace storeSerializer() {
+ return KryoNamespace.newBuilder()
+ .register(KryoNamespaces.API)
+ .register(TimedEntry.class)
+ .build();
+ }
+}
diff --git a/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/mirror/P4RuntimeGroupMirror.java b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/mirror/P4RuntimeGroupMirror.java
new file mode 100644
index 0000000..f363e71
--- /dev/null
+++ b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/mirror/P4RuntimeGroupMirror.java
@@ -0,0 +1,27 @@
+/*
+ * 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.p4runtime.mirror;
+
+import org.onosproject.net.pi.runtime.PiActionGroup;
+import org.onosproject.net.pi.runtime.PiActionGroupHandle;
+
+/**
+ * Mirror of action groups installed on a P4Runtime device.
+ */
+public interface P4RuntimeGroupMirror
+ extends P4RuntimeMirror<PiActionGroupHandle, PiActionGroup> {
+}
diff --git a/protocols/p4runtime/api/src/main/java/org/onosproject/p4runtime/api/P4RuntimeGroupReference.java b/protocols/p4runtime/api/src/main/java/org/onosproject/p4runtime/api/P4RuntimeGroupReference.java
deleted file mode 100644
index 55c9e61..0000000
--- a/protocols/p4runtime/api/src/main/java/org/onosproject/p4runtime/api/P4RuntimeGroupReference.java
+++ /dev/null
@@ -1,103 +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.p4runtime.api;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import org.onosproject.net.DeviceId;
-import org.onosproject.net.pi.model.PiActionProfileId;
-import org.onosproject.net.pi.runtime.PiActionGroupId;
-
-/**
- * Class containing the reference for a group in P4Runtime.
- */
-@Beta
-public final class P4RuntimeGroupReference {
- private final DeviceId deviceId;
- private final PiActionProfileId piActionProfileId;
- private final PiActionGroupId groupId;
-
- /**
- * Creates P4 runtime group reference.
- *
- * @param deviceId the device id of group
- * @param piActionProfileId the action profile id
- * @param groupId the group Id of group
- */
- public P4RuntimeGroupReference(DeviceId deviceId, PiActionProfileId piActionProfileId,
- PiActionGroupId groupId) {
- this.deviceId = deviceId;
- this.piActionProfileId = piActionProfileId;
- this.groupId = groupId;
- }
-
- /**
- * Gets device id of this group.
- *
- * @return the device id
- */
- public DeviceId deviceId() {
- return deviceId;
- }
-
- /**
- * Gets action profile id of this group.
- *
- * @return the action profile id
- */
- public PiActionProfileId actionProfileId() {
- return piActionProfileId;
- }
-
- /**
- * Gets group id of this group.
- *
- * @return group id
- */
- public PiActionGroupId groupId() {
- return groupId;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- P4RuntimeGroupReference that = (P4RuntimeGroupReference) o;
- return Objects.equal(deviceId, that.deviceId) &&
- Objects.equal(piActionProfileId, that.piActionProfileId) &&
- Objects.equal(groupId, that.groupId);
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(deviceId, piActionProfileId, groupId);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(this)
- .add("deviceId", deviceId)
- .add("piActionProfileId", piActionProfileId)
- .add("groupId", groupId)
- .toString();
- }
-}
diff --git a/protocols/p4runtime/api/src/main/java/org/onosproject/p4runtime/api/P4RuntimeGroupWrapper.java b/protocols/p4runtime/api/src/main/java/org/onosproject/p4runtime/api/P4RuntimeGroupWrapper.java
deleted file mode 100644
index 8cb4b83..0000000
--- a/protocols/p4runtime/api/src/main/java/org/onosproject/p4runtime/api/P4RuntimeGroupWrapper.java
+++ /dev/null
@@ -1,72 +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.p4runtime.api;
-
-import com.google.common.annotations.Beta;
-import org.onosproject.net.group.Group;
-import org.onosproject.net.pi.runtime.PiActionGroup;
-
-/**
- * A wrapper for a ONOS group installed on a P4Runtime device.
- */
-@Beta
-public class P4RuntimeGroupWrapper {
- private final PiActionGroup piActionGroup;
- private final Group group;
- private final long installMilliSeconds;
-
- /**
- * Creates new group wrapper.
- *
- * @param piActionGroup the Pi action group
- * @param group the group
- * @param installMilliSeconds the installation time
- */
- public P4RuntimeGroupWrapper(PiActionGroup piActionGroup, Group group,
- long installMilliSeconds) {
- this.piActionGroup = piActionGroup;
- this.group = group;
- this.installMilliSeconds = installMilliSeconds;
- }
-
- /**
- * Gets PI action group from this wrapper.
- *
- * @return the PI action group
- */
- public PiActionGroup piActionGroup() {
- return piActionGroup;
- }
-
- /**
- * Gets group from this wrapper.
- *
- * @return the group
- */
- public Group group() {
- return group;
- }
-
- /**
- * Gets installation time of this wrapper.
- *
- * @return the installation time
- */
- public long installMilliSeconds() {
- return installMilliSeconds;
- }
-}