[ONOS-6935] ActionProfile supports in P4RuntimeClient

Change-Id: I9f0ac307985c03b7ed93e14e41ba468c481a4e4f
diff --git a/drivers/p4runtime/BUCK b/drivers/p4runtime/BUCK
index 57079be..5c50f12 100644
--- a/drivers/p4runtime/BUCK
+++ b/drivers/p4runtime/BUCK
@@ -5,14 +5,22 @@
     '//protocols/p4runtime/api:onos-protocols-p4runtime-api',
     '//incubator/grpc-dependencies:grpc-core-repkg-' + GRPC_VER,
     '//lib:grpc-netty-' + GRPC_VER,
+    '//core/store/serializers:onos-core-serializers',
+    '//lib:KRYO',
 ]
 
 BUNDLES = [
     ':onos-drivers-p4runtime',
 ]
 
-osgi_jar(
+TEST_DEPS = [
+    '//lib:TEST_ADAPTERS',
+    '//core/api:onos-api-tests',
+]
+
+osgi_jar_with_tests (
     deps = COMPILE_DEPS,
+    test_deps = TEST_DEPS,
 )
 
 onos_app (
diff --git a/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/DefaultP4Interpreter.java b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/DefaultP4Interpreter.java
index e2fe315..d069e2a 100644
--- a/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/DefaultP4Interpreter.java
+++ b/drivers/p4runtime/src/main/java/org/onosproject/drivers/p4runtime/DefaultP4Interpreter.java
@@ -30,6 +30,7 @@
 import org.onosproject.net.flow.criteria.Criterion;
 import org.onosproject.net.flow.instructions.Instruction;
 import org.onosproject.net.flow.instructions.Instructions;
+import org.onosproject.net.flow.instructions.PiInstruction;
 import org.onosproject.net.packet.DefaultInboundPacket;
 import org.onosproject.net.packet.InboundPacket;
 import org.onosproject.net.packet.OutboundPacket;
@@ -167,7 +168,7 @@
 
     @Override
     public PiAction mapTreatment(TrafficTreatment treatment, PiTableId piTableId) throws PiInterpreterException {
-
+        initTargetSpecificAttributes();
         if (treatment.allInstructions().size() == 0) {
             // No instructions means drop for us.
             return actionWithName(DROP);
@@ -181,17 +182,10 @@
         switch (instruction.type()) {
             case OUTPUT:
                 Instructions.OutputInstruction outInstruction = (Instructions.OutputInstruction) instruction;
-                PortNumber port = outInstruction.port();
-                if (!port.isLogical()) {
-                    return PiAction.builder()
-                            .withId(PiActionId.of(SET_EGRESS_PORT))
-                            .withParameter(new PiActionParam(PiActionParamId.of(PORT), copyFrom(port.toLong())))
-                            .build();
-                } else if (port.equals(CONTROLLER)) {
-                    return actionWithName(SEND_TO_CPU);
-                } else {
-                    throw new PiInterpreterException(format("Egress on logical port '%s' not supported", port));
-                }
+                return outputPiAction(outInstruction);
+            case PROTOCOL_INDEPENDENT:
+                PiInstruction piInstruction = (PiInstruction) instruction;
+                return (PiAction) piInstruction.action();
             case NOACTION:
                 return actionWithName(DROP);
             default:
@@ -199,6 +193,25 @@
         }
     }
 
+    private PiAction outputPiAction(Instructions.OutputInstruction outInstruction) throws PiInterpreterException {
+        PortNumber port = outInstruction.port();
+        if (!port.isLogical()) {
+            try {
+                return PiAction.builder()
+                        .withId(PiActionId.of(SET_EGRESS_PORT))
+                        .withParameter(new PiActionParam(PiActionParamId.of(PORT),
+                                                         fit(copyFrom(port.toLong()), portFieldBitWidth)))
+                        .build();
+            } catch (ImmutableByteSequence.ByteSequenceTrimException e) {
+                throw new PiInterpreterException(e.getMessage());
+            }
+        } else if (port.equals(CONTROLLER)) {
+            return actionWithName(SEND_TO_CPU);
+        } else {
+            throw new PiInterpreterException(format("Egress on logical port '%s' not supported", port));
+        }
+    }
+
     @Override
     public Optional<PiCounterId> mapTableCounter(PiTableId piTableId) {
         return Optional.ofNullable(TABLE_COUNTER_MAP.get(piTableId));
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 15a40ca..e108123 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
@@ -16,71 +16,272 @@
 
 package org.onosproject.drivers.p4runtime;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.onlab.util.KryoNamespace;
+import org.onosproject.core.GroupId;
 import org.onosproject.net.Device;
 import org.onosproject.net.DeviceId;
-import org.onosproject.net.driver.AbstractHandlerBehaviour;
+import org.onosproject.net.device.DeviceService;
+import org.onosproject.net.flow.TrafficTreatment;
+import org.onosproject.net.flow.instructions.Instruction;
+import org.onosproject.net.flow.instructions.PiInstruction;
+import org.onosproject.net.group.Group;
 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.group.GroupStore;
 import org.onosproject.net.pi.model.PiPipelineInterpreter;
+import org.onosproject.net.pi.runtime.PiActionProfileId;
 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.PiTableAction;
 import org.onosproject.net.pi.runtime.PiTableId;
+import org.onosproject.p4runtime.api.P4RuntimeClient;
+import org.onosproject.p4runtime.api.P4RuntimeGroupReference;
+import org.onosproject.p4runtime.api.P4RuntimeGroupWrapper;
+import org.onosproject.store.serializers.KryoNamespaces;
+import org.slf4j.Logger;
 
 import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+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;
 
-public class P4RuntimeGroupProgrammable extends AbstractHandlerBehaviour implements GroupProgrammable {
+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";
+    private static final Logger log = getLogger(P4RuntimeGroupProgrammable.class);
+    private static final int GROUP_ID_MASK = 0xffff;
+    public static final KryoNamespace KRYO = new KryoNamespace.Builder()
+            .register(KryoNamespaces.API)
+            .register(DefaultP4RuntimeGroupCookie.class)
+            .build("P4RuntimeGroupProgrammable");
 
     /*
-    Work in progress.
+     * 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).
      */
 
-    private Device device;
-
+    // TODO: make this attribute configurable by child drivers (e.g. BMv2 or Tofino)
     /*
-    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).
+    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;
+
+    // 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 PiPipelineInterpreter interpreter;
+
+    protected boolean init() {
+        if (!setupBehaviour()) {
+            return false;
+        }
+        Device device = deviceService.getDevice(deviceId);
+        // Need an interpreter to map the bucket treatment to a PI action
+        if (!device.is(PiPipelineInterpreter.class)) {
+            log.warn("Can't find interpreter for device {}", device.id());
+        } else {
+            interpreter = device.as(PiPipelineInterpreter.class);
+        }
+        return true;
+    }
 
     @Override
     public void performGroupOperation(DeviceId deviceId, GroupOperations groupOps) {
+        if (!init()) {
+            // Ignore group operation of not initialized.
+            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);
+
+        // 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);
+
+        switch (group.type()) {
+            case SELECT:
+                piActionGroupBuilder.withType(PiActionGroup.Type.SELECT);
+                break;
+            default:
+                log.warn("Group type {} not supported, ignore group {}.", group.type(), groupId);
+                return;
+        }
+        /*
+            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:
+            - Add table information into app cookie and put into group description
+         */
+        // TODO: notify group service if we get deserialize error
+        DefaultP4RuntimeGroupCookie defaultP4RuntimeGroupCookie = KRYO.deserialize(group.appCookie().key());
+        PiTableId piTableId = defaultP4RuntimeGroupCookie.tableId();
+        PiActionProfileId piActionProfileId = defaultP4RuntimeGroupCookie.actionProfileId();
+        piActionGroupBuilder.withActionProfileId(piActionProfileId);
+
+        List<PiActionGroupMember> members = buildMembers(group, piActionGroupId, piTableId);
+        if (members == null) {
+            log.warn("Can't build members for group {} on {}", group, device.id());
+            return;
+        }
+
+        piActionGroupBuilder.addMembers(members);
+        PiActionGroup piActionGroup = piActionGroupBuilder.build();
+
+        P4RuntimeGroupReference groupRef =
+                new P4RuntimeGroupReference(deviceId, piActionProfileId, piActionGroupId);
+        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());
+            boolean success;
             switch (groupOp.opType()) {
                 case ADD:
-                    addGroup(deviceId, groupOp);
+                case MODIFY:
+                    success = writeGroupToDevice(oldGroupWrapper, piActionGroup, members);
+                    if (success) {
+                        GROUP_STORE.put(groupRef, newGroupWrapper);
+                    }
+                    break;
+                case DELETE:
+                    success = deleteGroupFromDevice(piActionGroup, members);
+                    if (success) {
+                        GROUP_STORE.remove(groupRef);
+                    }
                     break;
                 default:
                     throw new UnsupportedOperationException();
             }
+        } finally {
+            lock.unlock();
         }
     }
 
-    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
+    /**
+     * 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
+     * @param members members of the action group
+     * @return true if install success; false otherwise
+     */
+    private boolean writeGroupToDevice(P4RuntimeGroupWrapper oldGroupWrapper,
+                                       PiActionGroup piActionGroup,
+                                       Collection<PiActionGroupMember> members) {
+        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(),
+                                            oldGroupWrapper.piActionGroup().members());
+        }
+        writeSuccess = client.writeActionGroupMembers(piActionGroup,
+                                                      members,
+                                                      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,
+                                          Collection<PiActionGroupMember> members) {
+        boolean success;
+        CompletableFuture<Boolean> writeSuccess;
+        writeSuccess = client.writeActionGroup(piActionGroup,
+                                P4RuntimeClient.WriteOperationType.DELETE,
+                                pipeconf);
+        success = completeSuccess(writeSuccess, ACT_GRP, DELETE);
+        writeSuccess = client.writeActionGroupMembers(piActionGroup,
+                                       members,
+                                       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;
+        }
+    }
+
+    /**
+     * Build pi action group members from group.
+     *
+     * @param group the group
+     * @param piActionGroupId the PI action group id of the group
+     * @param piTableId the PI table related to the group
+     * @return list of PI action group members; null if can't build member list
+     */
+    private List<PiActionGroupMember> buildMembers(Group group, PiActionGroupId piActionGroupId, PiTableId piTableId) {
+        GroupId groupId = group.id();
+        ImmutableList.Builder<PiActionGroupMember> membersBuilder = ImmutableList.builder();
 
         int bucketIdx = 0;
-        for (GroupBucket bucket : groupOp.buckets().buckets()) {
+        for (GroupBucket bucket : group.buckets().buckets()) {
             /*
             Problem:
             In P4Runtime action group members, i.e. action buckets, are associated to a numeric ID chosen
@@ -95,40 +296,46 @@
             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();
+            ByteBuffer bb = ByteBuffer.allocate(4)
+                    .putShort((short) (piActionGroupId.id() & GROUP_ID_MASK))
+                    .putShort((short) bucketIdx);
+            bb.rewind();
+            int memberId = bb.getInt();
 
-            // Need an interpreter to map the bucket treatment to a PI action
+            bucketIdx++;
+            PiAction action;
+            if (interpreter != null) {
+                // if we have interpreter, use interpreter
+                try {
+                    action = interpreter.mapTreatment(bucket.treatment(), piTableId);
+                } catch (PiPipelineInterpreter.PiInterpreterException e) {
+                    log.warn("Can't map treatment {} to action due to {}, ignore group {}",
+                             bucket.treatment(), e.getMessage(), groupId);
+                    return null;
+                }
+            } else {
+                // if we don't have interpreter, accept PiInstruction only
+                TrafficTreatment treatment = bucket.treatment();
 
-            if (!device.is(PiPipelineInterpreter.class)) {
-                // log error
-            }
+                if (treatment.allInstructions().size() > 1) {
+                    log.warn("Treatment {} has multiple instructions, ignore group {}",
+                             treatment, groupId);
+                    return null;
+                }
+                Instruction instruction = treatment.allInstructions().get(0);
+                if (instruction.type() != Instruction.Type.PROTOCOL_INDEPENDENT) {
+                    log.warn("Instruction {} is not a PROTOCOL_INDEPENDENT type, ignore group {}",
+                             instruction, groupId);
+                    return null;
+                }
 
-            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
+                PiInstruction piInstruction = (PiInstruction) instruction;
+                if (piInstruction.action().type() != PiTableAction.Type.ACTION) {
+                    log.warn("Action {} is not an ACTION type, ignore group {}",
+                             piInstruction.action(), groupId);
+                    return null;
+                }
+                action = (PiAction) piInstruction.action();
             }
 
             PiActionGroupMember member = PiActionGroupMember.builder()
@@ -137,15 +344,97 @@
                     .withWeight(bucket.weight())
                     .build();
 
-            piActionGroupBuilder.addMember(member);
+            membersBuilder.add(member);
+        }
+        return membersBuilder.build();
+    }
 
-            // Use P4RuntimeClient to install member;
-            // TODO: implement P4RuntimeClient method.
+    @Override
+    public Collection<Group> getGroups() {
+        if (!init()) {
+            return Collections.emptySet();
         }
 
-        PiActionGroup piActionGroup = piActionGroupBuilder.build();
+        Collection<Group> result = Sets.newHashSet();
+        Collection<PiActionProfileId> piActionProfileIds = Sets.newHashSet();
 
-        // Use P4RuntimeClient to insert group.
-        // TODO: implement P4RuntimeClient method.
+        // Collection action profile Ids
+        // TODO: find better way to get all action profile ids....
+        GROUP_STORE.forEach((groupRef, wrapper) -> piActionProfileIds.add(groupRef.actionProfileId()));
+
+        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);
+            }
+
+            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;
+        }
+    }
+
+    /**
+     * P4Runtime app cookie for group.
+     */
+    public static class DefaultP4RuntimeGroupCookie {
+        private PiTableId tableId;
+        private PiActionProfileId piActionProfileId;
+        private Integer groupId;
+
+        public DefaultP4RuntimeGroupCookie(PiTableId tableId,
+                                           PiActionProfileId piActionProfileId,
+                                           Integer groupId) {
+            this.tableId = tableId;
+            this.piActionProfileId = piActionProfileId;
+            this.groupId = groupId;
+        }
+
+        public PiTableId tableId() {
+            return tableId;
+        }
+
+        public PiActionProfileId actionProfileId() {
+            return piActionProfileId;
+        }
+
+        public Integer groupId() {
+            return groupId;
+        }
     }
 }
diff --git a/drivers/p4runtime/src/test/java/org/onosproject/drivers/p4runtime/P4runtimeGroupProgrammableTest.java b/drivers/p4runtime/src/test/java/org/onosproject/drivers/p4runtime/P4runtimeGroupProgrammableTest.java
new file mode 100644
index 0000000..c334218
--- /dev/null
+++ b/drivers/p4runtime/src/test/java/org/onosproject/drivers/p4runtime/P4runtimeGroupProgrammableTest.java
@@ -0,0 +1,291 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import org.easymock.Capture;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.onlab.util.ImmutableByteSequence;
+import org.onosproject.TestApplicationId;
+import org.onosproject.core.ApplicationId;
+import org.onosproject.core.GroupId;
+import org.onosproject.drivers.p4runtime.P4RuntimeGroupProgrammable.DefaultP4RuntimeGroupCookie;
+import org.onosproject.net.Device;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.device.DeviceService;
+import org.onosproject.net.driver.DriverData;
+import org.onosproject.net.driver.DriverHandler;
+import org.onosproject.net.flow.DefaultTrafficTreatment;
+import org.onosproject.net.flow.TrafficTreatment;
+import org.onosproject.net.flow.instructions.Instructions;
+import org.onosproject.net.group.DefaultGroup;
+import org.onosproject.net.group.DefaultGroupBucket;
+import org.onosproject.net.group.DefaultGroupDescription;
+import org.onosproject.net.group.DefaultGroupKey;
+import org.onosproject.net.group.Group;
+import org.onosproject.net.group.GroupBucket;
+import org.onosproject.net.group.GroupBuckets;
+import org.onosproject.net.group.GroupDescription;
+import org.onosproject.net.group.GroupKey;
+import org.onosproject.net.group.GroupOperation;
+import org.onosproject.net.group.GroupOperations;
+import org.onosproject.net.group.GroupStore;
+import org.onosproject.net.pi.model.DefaultPiPipeconf;
+import org.onosproject.net.pi.model.PiPipeconf;
+import org.onosproject.net.pi.model.PiPipeconfId;
+import org.onosproject.net.pi.model.PiPipelineInterpreter;
+import org.onosproject.net.pi.model.PiPipelineModel;
+import org.onosproject.net.pi.runtime.PiActionProfileId;
+import org.onosproject.net.pi.runtime.PiAction;
+import org.onosproject.net.pi.runtime.PiActionGroup;
+import org.onosproject.net.pi.runtime.PiActionGroupMember;
+import org.onosproject.net.pi.runtime.PiActionGroupMemberId;
+import org.onosproject.net.pi.runtime.PiActionId;
+import org.onosproject.net.pi.runtime.PiActionParam;
+import org.onosproject.net.pi.runtime.PiActionParamId;
+import org.onosproject.net.pi.runtime.PiPipeconfService;
+import org.onosproject.net.pi.runtime.PiTableAction;
+import org.onosproject.net.pi.runtime.PiTableId;
+import org.onosproject.p4runtime.api.P4RuntimeClient;
+import org.onosproject.p4runtime.api.P4RuntimeController;
+
+import java.net.URL;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import static org.easymock.EasyMock.*;
+import static org.junit.Assert.assertEquals;
+import static org.onosproject.net.group.GroupDescription.Type.SELECT;
+import static org.onosproject.net.pi.model.PiPipeconf.ExtensionType.P4_INFO_TEXT;
+import static org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType.DELETE;
+import static org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType.INSERT;
+
+public class P4runtimeGroupProgrammableTest {
+    private static final String P4INFO_PATH = "/default.p4info";
+    private static final DeviceId DEVICE_ID = DeviceId.deviceId("device:p4runtime:1");
+    private static final PiPipeconfId PIPECONF_ID = new PiPipeconfId("p4runtime-mock-pipeconf");
+    private static final PiPipeconf PIPECONF = buildPipeconf();
+    private static final PiTableId ECMP_TABLE_ID = PiTableId.of("ecmp");
+    private static final PiActionProfileId ACT_PROF_ID = PiActionProfileId.of("ecmp_selector");
+    private static final ApplicationId APP_ID = TestApplicationId.create("P4runtimeGroupProgrammableTest");
+    private static final GroupId GROUP_ID = GroupId.valueOf(1);
+    private static final PiActionId EGRESS_PORT_ACTION_ID = PiActionId.of("set_egress_port");
+    private static final PiActionParamId PORT_PARAM_ID = PiActionParamId.of("port");
+    private static final List<GroupBucket> BUCKET_LIST = ImmutableList.of(
+            outputBucket(1),
+            outputBucket(2),
+            outputBucket(3)
+    );
+    private static final DefaultP4RuntimeGroupCookie COOKIE =
+            new DefaultP4RuntimeGroupCookie(ECMP_TABLE_ID, ACT_PROF_ID, GROUP_ID.id());
+    private static final GroupKey GROUP_KEY =
+            new DefaultGroupKey(P4RuntimeGroupProgrammable.KRYO.serialize(COOKIE));
+    private static final GroupBuckets BUCKETS = new GroupBuckets(BUCKET_LIST);
+    private static final GroupDescription GROUP_DESC =
+            new DefaultGroupDescription(DEVICE_ID,
+                                        SELECT,
+                                        BUCKETS,
+                                        GROUP_KEY,
+                                        GROUP_ID.id(),
+                                        APP_ID);
+    private static final Group GROUP = new DefaultGroup(GROUP_ID, GROUP_DESC);
+    private static final int DEFAULT_MEMBER_WEIGHT = 1;
+    private static final int BASE_MEM_ID = 65535;
+    private static final Collection<PiActionGroupMember> EXPECTED_MEMBERS =
+            ImmutableSet.of(
+                    outputMember(1),
+                    outputMember(2),
+                    outputMember(3)
+            );
+
+    private P4RuntimeGroupProgrammable programmable;
+    private DriverHandler driverHandler;
+    private DriverData driverData;
+    private P4RuntimeController controller;
+    private P4RuntimeClient client;
+    private PiPipeconfService piPipeconfService;
+    private DeviceService deviceService;
+    private Device device;
+    private GroupStore groupStore;
+
+    private static PiPipeconf buildPipeconf() {
+        final URL p4InfoUrl = P4runtimeGroupProgrammableTest.class.getResource(P4INFO_PATH);
+        return DefaultPiPipeconf.builder()
+                .withId(PIPECONF_ID)
+                .withPipelineModel(niceMock(PiPipelineModel.class))
+                .addExtension(P4_INFO_TEXT, p4InfoUrl)
+                .build();
+    }
+
+    private static GroupBucket outputBucket(int portNum) {
+        ImmutableByteSequence paramVal = ImmutableByteSequence.copyFrom(portNum);
+        PiActionParam param = new PiActionParam(PiActionParamId.of(PORT_PARAM_ID.name()), paramVal);
+        PiTableAction action = PiAction.builder().withId(EGRESS_PORT_ACTION_ID).withParameter(param).build();
+
+        TrafficTreatment treatment = DefaultTrafficTreatment.builder()
+                .add(Instructions.piTableAction(action))
+                .build();
+
+        return DefaultGroupBucket.createSelectGroupBucket(treatment);
+    }
+
+    private static PiActionGroupMember outputMember(int portNum) {
+        PiActionParam param = new PiActionParam(PORT_PARAM_ID,
+                                                ImmutableByteSequence.copyFrom(portNum));
+        PiAction piAction = PiAction.builder()
+                .withId(EGRESS_PORT_ACTION_ID)
+                .withParameter(param).build();
+
+        return PiActionGroupMember.builder()
+                .withAction(piAction)
+                .withId(PiActionGroupMemberId.of(BASE_MEM_ID + portNum))
+                .withWeight(DEFAULT_MEMBER_WEIGHT)
+                .build();
+    }
+
+    @Before
+    public void setup() {
+        driverHandler = EasyMock.niceMock(DriverHandler.class);
+        driverData = EasyMock.niceMock(DriverData.class);
+        controller = EasyMock.niceMock(P4RuntimeController.class);
+        client = EasyMock.niceMock(P4RuntimeClient.class);
+        piPipeconfService = EasyMock.niceMock(PiPipeconfService.class);
+        deviceService = EasyMock.niceMock(DeviceService.class);
+        device = EasyMock.niceMock(Device.class);
+        groupStore = EasyMock.niceMock(GroupStore.class);
+
+        expect(controller.hasClient(DEVICE_ID)).andReturn(true).anyTimes();
+        expect(controller.getClient(DEVICE_ID)).andReturn(client).anyTimes();
+        expect(device.is(PiPipelineInterpreter.class)).andReturn(true).anyTimes();
+        expect(device.id()).andReturn(DEVICE_ID).anyTimes();
+        expect(deviceService.getDevice(DEVICE_ID)).andReturn(device).anyTimes();
+        expect(driverData.deviceId()).andReturn(DEVICE_ID).anyTimes();
+        expect(groupStore.getGroup(DEVICE_ID, GROUP_ID)).andReturn(GROUP).anyTimes();
+        expect(piPipeconfService.ofDevice(DEVICE_ID)).andReturn(Optional.of(PIPECONF_ID)).anyTimes();
+        expect(piPipeconfService.getPipeconf(PIPECONF_ID)).andReturn(Optional.of(PIPECONF)).anyTimes();
+        expect(driverHandler.data()).andReturn(driverData).anyTimes();
+        expect(driverHandler.get(P4RuntimeController.class)).andReturn(controller).anyTimes();
+        expect(driverHandler.get(PiPipeconfService.class)).andReturn(piPipeconfService).anyTimes();
+        expect(driverHandler.get(DeviceService.class)).andReturn(deviceService).anyTimes();
+        expect(driverHandler.get(GroupStore.class)).andReturn(groupStore).anyTimes();
+
+        programmable = new P4RuntimeGroupProgrammable();
+        programmable.setHandler(driverHandler);
+        programmable.setData(driverData);
+        EasyMock.replay(driverHandler, driverData, controller, piPipeconfService,
+                        deviceService, device, groupStore);
+    }
+
+    /**
+     * Test init function.
+     */
+    @Test
+    public void testInit() {
+        programmable.init();
+    }
+
+    /**
+     * Test add group with buckets.
+     */
+    @Test
+    public void testAddGroup() {
+        List<GroupOperation> ops = Lists.newArrayList();
+        ops.add(GroupOperation.createAddGroupOperation(GROUP_ID, SELECT, BUCKETS));
+        GroupOperations groupOps = new GroupOperations(ops);
+        CompletableFuture<Boolean> completeTrue = new CompletableFuture<>();
+        completeTrue.complete(true);
+
+        Capture<PiActionGroup> groupCapture1 = EasyMock.newCapture();
+        expect(client.writeActionGroup(EasyMock.capture(groupCapture1), EasyMock.eq(INSERT), EasyMock.eq(PIPECONF)))
+                .andReturn(completeTrue).anyTimes();
+
+        Capture<PiActionGroup> groupCapture2 = EasyMock.newCapture();
+        Capture<Collection<PiActionGroupMember>> membersCapture = EasyMock.newCapture();
+        expect(client.writeActionGroupMembers(EasyMock.capture(groupCapture2),
+                                                       EasyMock.capture(membersCapture),
+                                                       EasyMock.eq(INSERT),
+                                                       EasyMock.eq(PIPECONF)))
+                .andReturn(completeTrue).anyTimes();
+
+        EasyMock.replay(client);
+        programmable.performGroupOperation(DEVICE_ID, groupOps);
+
+        // verify group installed by group programmable
+        PiActionGroup group1 = groupCapture1.getValue();
+        PiActionGroup group2 = groupCapture2.getValue();
+        assertEquals("Groups should be equal", group1, group2);
+        assertEquals(GROUP_ID.id(), group1.id().id());
+        assertEquals(PiActionGroup.Type.SELECT, group1.type());
+        assertEquals(ACT_PROF_ID, group1.actionProfileId());
+
+        // members installed
+        Collection<PiActionGroupMember> members = group1.members();
+        assertEquals(3, members.size());
+
+        Assert.assertTrue(EXPECTED_MEMBERS.containsAll(members));
+        Assert.assertTrue(members.containsAll(EXPECTED_MEMBERS));
+    }
+
+    /**
+     * Test remove group with buckets.
+     */
+    @Test
+    public void testDelGroup() {
+        List<GroupOperation> ops = Lists.newArrayList();
+        ops.add(GroupOperation.createDeleteGroupOperation(GROUP_ID, SELECT));
+        GroupOperations groupOps = new GroupOperations(ops);
+        CompletableFuture<Boolean> completeTrue = new CompletableFuture<>();
+        completeTrue.complete(true);
+
+        Capture<PiActionGroup> groupCapture1 = EasyMock.newCapture();
+        expect(client.writeActionGroup(EasyMock.capture(groupCapture1), EasyMock.eq(DELETE), EasyMock.eq(PIPECONF)))
+                .andReturn(completeTrue).anyTimes();
+
+        Capture<PiActionGroup> groupCapture2 = EasyMock.newCapture();
+        Capture<Collection<PiActionGroupMember>> membersCapture = EasyMock.newCapture();
+        expect(client.writeActionGroupMembers(EasyMock.capture(groupCapture2),
+                                                       EasyMock.capture(membersCapture),
+                                                       EasyMock.eq(DELETE),
+                                                       EasyMock.eq(PIPECONF)))
+                .andReturn(completeTrue).anyTimes();
+
+        EasyMock.replay(client);
+        programmable.performGroupOperation(DEVICE_ID, groupOps);
+
+        // verify group installed by group programmable
+        PiActionGroup group1 = groupCapture1.getValue();
+        PiActionGroup group2 = groupCapture2.getValue();
+        assertEquals("Groups should be equal", group1, group2);
+        assertEquals(GROUP_ID.id(), group1.id().id());
+        assertEquals(PiActionGroup.Type.SELECT, group1.type());
+        assertEquals(ACT_PROF_ID, group1.actionProfileId());
+
+        // members installed
+        Collection<PiActionGroupMember> members = group1.members();
+        assertEquals(3, members.size());
+
+        Assert.assertTrue(EXPECTED_MEMBERS.containsAll(members));
+        Assert.assertTrue(members.containsAll(EXPECTED_MEMBERS));
+    }
+}
diff --git a/drivers/p4runtime/src/test/resources/default.p4info b/drivers/p4runtime/src/test/resources/default.p4info
new file mode 120000
index 0000000..8f71cbe
--- /dev/null
+++ b/drivers/p4runtime/src/test/resources/default.p4info
@@ -0,0 +1 @@
+../../../../../tools/test/p4src/p4-16/p4c-out/default.p4info
\ No newline at end of file