[ONOS-6935] ActionProfile supports in P4RuntimeClient

Change-Id: I9f0ac307985c03b7ed93e14e41ba468c481a4e4f
diff --git a/protocols/p4runtime/ctl/BUCK b/protocols/p4runtime/ctl/BUCK
index 899a1ff..cb22402 100644
--- a/protocols/p4runtime/ctl/BUCK
+++ b/protocols/p4runtime/ctl/BUCK
@@ -13,8 +13,9 @@
 
 TEST_DEPS = [
     '//lib:TEST',
+    '//lib:GRPC_TEST_1.3',
     '//lib:minimal-json',
-    '//incubator/bmv2/model:onos-incubator-bmv2-model',
+    '//lib:grpc-protobuf-lite-' + GRPC_VER,
 ]
 
 osgi_jar_with_tests(
diff --git a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/ActionProfileGroupEncoder.java b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/ActionProfileGroupEncoder.java
new file mode 100644
index 0000000..472bf8e
--- /dev/null
+++ b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/ActionProfileGroupEncoder.java
@@ -0,0 +1,144 @@
+/*
+ * 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.ctl;
+
+import com.google.common.collect.Maps;
+import org.onosproject.net.pi.model.PiPipeconf;
+import org.onosproject.net.pi.runtime.PiActionProfileId;
+import org.onosproject.net.pi.runtime.PiActionGroup;
+import org.onosproject.net.pi.runtime.PiActionGroupId;
+import p4.P4RuntimeOuterClass.ActionProfileGroup;
+import p4.P4RuntimeOuterClass.ActionProfileGroup.Member;
+import p4.P4RuntimeOuterClass.ActionProfileMember;
+import p4.config.P4InfoOuterClass;
+
+import java.util.Collection;
+import java.util.Map;
+
+import static java.lang.String.format;
+
+/**
+ * Encoder/Decoder for action profile group.
+ */
+public final class ActionProfileGroupEncoder {
+    private ActionProfileGroupEncoder() {
+        // hide default constructor
+    }
+
+    /**
+     * Encode a PI action group to a action profile group.
+     *
+     * @param piActionGroup the action profile group
+     * @param pipeconf the pipeconf
+     * @return a action profile group encoded from PI action group
+     * @throws P4InfoBrowser.NotFoundException if can't find action profile from P4Info browser
+     * @throws EncodeException if can't find P4Info from pipeconf
+     */
+    static ActionProfileGroup encode(PiActionGroup piActionGroup, PiPipeconf pipeconf)
+            throws P4InfoBrowser.NotFoundException, EncodeException {
+        P4InfoBrowser browser = PipeconfHelper.getP4InfoBrowser(pipeconf);
+
+        if (browser == null) {
+            throw new EncodeException(format("Can't get P4 info browser from pipeconf %s", pipeconf));
+        }
+
+        PiActionProfileId piActionProfileId = piActionGroup.actionProfileId();
+        int actionProfileId;
+        P4InfoOuterClass.ActionProfile actionProfile =
+                browser.actionProfiles().getByName(piActionProfileId.id());
+        actionProfileId = actionProfile.getPreamble().getId();
+        ActionProfileGroup.Builder actionProfileGroupBuilder =
+                ActionProfileGroup.newBuilder()
+                        .setGroupId(piActionGroup.id().id())
+                        .setActionProfileId(actionProfileId);
+
+        switch (piActionGroup.type()) {
+            case SELECT:
+                actionProfileGroupBuilder.setType(ActionProfileGroup.Type.SELECT);
+                break;
+            default:
+                throw new EncodeException(format("Unsupported pi action group type %s",
+                                                 piActionGroup.type()));
+        }
+
+        piActionGroup.members().forEach(m -> {
+            // TODO: currently we don't set "watch" field of member
+            Member member = Member.newBuilder()
+                    .setMemberId(m.id().id())
+                    .setWeight(m.weight())
+                    .build();
+            actionProfileGroupBuilder.addMembers(member);
+        });
+
+        return actionProfileGroupBuilder.build();
+    }
+
+    /**
+     * Decode an action profile group with members information to a PI action group.
+     *
+     * @param actionProfileGroup the action profile group
+     * @param members members of the action profile group
+     * @param pipeconf the pipeconf
+     * @return decoded PI action group
+     * @throws P4InfoBrowser.NotFoundException if can't find action profile from P4Info browser
+     * @throws EncodeException if can't find P4Info from pipeconf
+     */
+    static PiActionGroup decode(ActionProfileGroup actionProfileGroup,
+                                        Collection<ActionProfileMember> members,
+                                        PiPipeconf pipeconf)
+            throws P4InfoBrowser.NotFoundException, EncodeException {
+        P4InfoBrowser browser = PipeconfHelper.getP4InfoBrowser(pipeconf);
+        if (browser == null) {
+            throw new EncodeException(format("Can't get P4 info browser from pipeconf %s", pipeconf));
+        }
+        PiActionGroup.Builder piActionGroupBuilder = PiActionGroup.builder();
+
+        P4InfoOuterClass.ActionProfile actionProfile = browser.actionProfiles()
+                .getById(actionProfileGroup.getActionProfileId());
+        PiActionProfileId piActionProfileId = PiActionProfileId.of(actionProfile.getPreamble().getName());
+        piActionGroupBuilder.withActionProfileId(piActionProfileId)
+                .withId(PiActionGroupId.of(actionProfileGroup.getGroupId()));
+
+        switch (actionProfileGroup.getType()) {
+            case SELECT:
+                piActionGroupBuilder.withType(PiActionGroup.Type.SELECT);
+                break;
+            default:
+                throw new EncodeException(format("Unsupported action profile type %s",
+                                                 actionProfileGroup.getType()));
+        }
+
+        Map<Integer, Integer> memberWeights = Maps.newHashMap();
+        actionProfileGroup.getMembersList().forEach(member -> {
+            int weight = member.getWeight();
+            if (weight < 1) {
+                // FIXME: currently PI has a bug which will always return weight 0
+                // ONOS won't accept group member with weight 0
+                weight = 1;
+            }
+            memberWeights.put(member.getMemberId(), weight);
+        });
+
+        for (ActionProfileMember member : members) {
+            int weight = memberWeights.get(member.getMemberId());
+            piActionGroupBuilder
+                    .addMember(ActionProfileMemberEncoder.decode(member, weight, pipeconf));
+        }
+
+        return piActionGroupBuilder.build();
+    }
+}
diff --git a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/ActionProfileMemberEncoder.java b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/ActionProfileMemberEncoder.java
new file mode 100644
index 0000000..2f08a59
--- /dev/null
+++ b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/ActionProfileMemberEncoder.java
@@ -0,0 +1,112 @@
+/*
+ * 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.ctl;
+
+import org.onosproject.net.pi.model.PiPipeconf;
+import org.onosproject.net.pi.runtime.PiActionGroup;
+import org.onosproject.net.pi.runtime.PiActionGroupMember;
+import org.onosproject.net.pi.runtime.PiActionGroupMemberId;
+import p4.P4RuntimeOuterClass;
+import p4.P4RuntimeOuterClass.ActionProfileMember;
+import p4.config.P4InfoOuterClass;
+
+import static java.lang.String.format;
+import static org.onosproject.p4runtime.ctl.TableEntryEncoder.decodeActionMsg;
+import static org.onosproject.p4runtime.ctl.TableEntryEncoder.encodePiAction;
+
+/**
+ * Encoder/Decoder of action profile member.
+ */
+public final class ActionProfileMemberEncoder {
+    private ActionProfileMemberEncoder() {
+        // Hide default constructor
+    }
+
+    /**
+     * Encode a PiActionGroupMember to a ActionProfileMember.
+     *
+     * @param group the PI action group of members
+     * @param member the member to encode
+     * @param pipeconf the pipeconf
+     * @return encoded member
+     */
+    /**
+     * Encode a PiActionGroupMember to a ActionProfileMember.
+     *
+     * @param group the PI action group of members
+     * @param member the member to encode
+     * @param pipeconf the pipeconf, as encode spec
+     * @return encoded member
+     * @throws P4InfoBrowser.NotFoundException can't find action profile from P4Info browser
+     * @throws EncodeException can't find P4Info from pipeconf
+     */
+    static ActionProfileMember encode(PiActionGroup group,
+                                      PiActionGroupMember member,
+                                      PiPipeconf pipeconf)
+            throws P4InfoBrowser.NotFoundException, EncodeException {
+
+        P4InfoBrowser browser = PipeconfHelper.getP4InfoBrowser(pipeconf);
+
+        if (browser == null) {
+            throw new EncodeException(format("Can't get P4 info browser from pipeconf %s", pipeconf));
+        }
+
+        ActionProfileMember.Builder actionProfileMemberBuilder =
+                ActionProfileMember.newBuilder();
+
+        // member id
+        actionProfileMemberBuilder.setMemberId(member.id().id());
+
+        // action profile id
+        P4InfoOuterClass.ActionProfile actionProfile =
+                browser.actionProfiles().getByName(group.actionProfileId().id());
+
+        int actionProfileId = actionProfile.getPreamble().getId();
+        actionProfileMemberBuilder.setActionProfileId(actionProfileId);
+
+        // Action
+        P4RuntimeOuterClass.Action action = encodePiAction(member.action(), browser);
+        actionProfileMemberBuilder.setAction(action);
+
+        return actionProfileMemberBuilder.build();
+    }
+
+    /**
+     * Decode an action profile member to PI action group member.
+     *
+     * @param member the action profile member
+     * @param weight the weight of the member
+     * @param pipeconf the pipeconf, as decode spec
+     * @return decoded PI action group member
+     * @throws P4InfoBrowser.NotFoundException can't find definition of action from P4 info
+     * @throws EncodeException can't get P4 info browser from pipeconf
+     */
+    static PiActionGroupMember decode(ActionProfileMember member,
+                                      int weight,
+                                      PiPipeconf pipeconf)
+            throws P4InfoBrowser.NotFoundException, EncodeException {
+        P4InfoBrowser browser = PipeconfHelper.getP4InfoBrowser(pipeconf);
+
+        if (browser == null) {
+            throw new EncodeException(format("Can't get P4 info browser from pipeconf %s", pipeconf));
+        }
+        return PiActionGroupMember.builder().withId(PiActionGroupMemberId.of(member.getMemberId()))
+                .withWeight(weight)
+                .withAction(decodeActionMsg(member.getAction(), browser))
+                .build();
+    }
+}
diff --git a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeClientImpl.java b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeClientImpl.java
index a9adf55..7f22e79 100644
--- a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeClientImpl.java
+++ b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeClientImpl.java
@@ -17,7 +17,10 @@
 package org.onosproject.p4runtime.ctl;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 import com.google.protobuf.ByteString;
 import io.grpc.Context;
@@ -26,13 +29,17 @@
 import io.grpc.StatusRuntimeException;
 import io.grpc.stub.StreamObserver;
 import org.onlab.osgi.DefaultServiceDirectory;
+import org.onlab.util.Tools;
 import org.onosproject.net.DeviceId;
 import org.onosproject.net.pi.model.PiPipeconf;
+import org.onosproject.net.pi.runtime.PiActionProfileId;
 import org.onosproject.net.pi.runtime.PiCounterCellData;
 import org.onosproject.net.pi.runtime.PiCounterCellId;
 import org.onosproject.net.pi.runtime.PiCounterId;
 import org.onosproject.net.pi.runtime.PiDirectCounterCellId;
 import org.onosproject.net.pi.runtime.PiIndirectCounterCellId;
+import org.onosproject.net.pi.runtime.PiActionGroup;
+import org.onosproject.net.pi.runtime.PiActionGroupMember;
 import org.onosproject.net.pi.runtime.PiPacketOperation;
 import org.onosproject.net.pi.runtime.PiPipeconfService;
 import org.onosproject.net.pi.runtime.PiTableEntry;
@@ -41,6 +48,8 @@
 import org.onosproject.p4runtime.api.P4RuntimeEvent;
 import org.slf4j.Logger;
 import p4.P4RuntimeGrpc;
+import p4.P4RuntimeOuterClass.ActionProfileGroup;
+import p4.P4RuntimeOuterClass.ActionProfileMember;
 import p4.P4RuntimeOuterClass.Entity;
 import p4.P4RuntimeOuterClass.ForwardingPipelineConfig;
 import p4.P4RuntimeOuterClass.MasterArbitrationUpdate;
@@ -53,6 +62,7 @@
 import p4.P4RuntimeOuterClass.TableEntry;
 import p4.P4RuntimeOuterClass.Update;
 import p4.P4RuntimeOuterClass.WriteRequest;
+import p4.config.P4InfoOuterClass;
 import p4.config.P4InfoOuterClass.P4Info;
 import p4.tmp.P4Config;
 
@@ -69,6 +79,8 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Supplier;
@@ -78,6 +90,8 @@
 import static org.onlab.util.Tools.groupedThreads;
 import static org.onosproject.net.pi.model.PiPipeconf.ExtensionType;
 import static org.slf4j.LoggerFactory.getLogger;
+import static p4.P4RuntimeOuterClass.Entity.EntityCase.ACTION_PROFILE_GROUP;
+import static p4.P4RuntimeOuterClass.Entity.EntityCase.ACTION_PROFILE_MEMBER;
 import static p4.P4RuntimeOuterClass.Entity.EntityCase.TABLE_ENTRY;
 import static p4.P4RuntimeOuterClass.PacketOut;
 import static p4.P4RuntimeOuterClass.SetForwardingPipelineConfigRequest.Action.VERIFY_AND_COMMIT;
@@ -106,7 +120,14 @@
     private final Lock writeLock = new ReentrantLock();
     private final StreamObserver<StreamMessageRequest> streamRequestObserver;
 
-
+    /**
+     * Default constructor.
+     *
+     * @param deviceId the ONOS device id
+     * @param p4DeviceId the P4 device id
+     * @param channel gRPC channel
+     * @param controller runtime client controller
+     */
     P4RuntimeClientImpl(DeviceId deviceId, long p4DeviceId, ManagedChannel channel,
                         P4RuntimeControllerImpl controller) {
         this.deviceId = deviceId;
@@ -216,6 +237,30 @@
                                "readAllCounterCells-" + cellIds.hashCode());
     }
 
+    @Override
+    public CompletableFuture<Boolean> writeActionGroupMembers(PiActionGroup group,
+                                                              Collection<PiActionGroupMember> members,
+                                                              WriteOperationType opType,
+                                                              PiPipeconf pipeconf) {
+        return supplyInContext(() -> doWriteActionGroupMembers(group, members, opType, pipeconf),
+                               "writeActionGroupMembers-" + opType.name());
+    }
+
+    @Override
+    public CompletableFuture<Boolean> writeActionGroup(PiActionGroup group,
+                                                       WriteOperationType opType,
+                                                       PiPipeconf pipeconf) {
+        return supplyInContext(() -> doWriteActionGroup(group, opType, pipeconf),
+                               "writeActionGroup-" + opType.name());
+    }
+
+    @Override
+    public CompletableFuture<Collection<PiActionGroup>> dumpGroups(PiActionProfileId actionProfileId,
+                                                                   PiPipeconf pipeconf) {
+        return supplyInContext(() -> doDumpGroups(actionProfileId, pipeconf),
+                               "dumpGroups-" + actionProfileId.id());
+    }
+
     /* Blocking method implementations below */
 
     private boolean doInitStreamChannel() {
@@ -464,6 +509,191 @@
         return CounterEntryCodec.decodeCounterEntities(entities, counterIdMap, pipeconf);
     }
 
+    private boolean doWriteActionGroupMembers(PiActionGroup group, Collection<PiActionGroupMember> members,
+                                              WriteOperationType opType, PiPipeconf pipeconf) {
+        WriteRequest.Builder writeRequestBuilder = WriteRequest.newBuilder();
+
+        Collection<ActionProfileMember> actionProfileMembers = Lists.newArrayList();
+        try {
+            for (PiActionGroupMember member : members) {
+                actionProfileMembers.add(
+                        ActionProfileMemberEncoder.encode(group, member, pipeconf)
+                );
+            }
+        } catch (EncodeException | P4InfoBrowser.NotFoundException e) {
+            log.warn("Can't encode group member {} due to {}", members, e.getMessage());
+            return false;
+        }
+
+        Collection<Update> updateMsgs = actionProfileMembers.stream()
+                .map(actionProfileMember ->
+                             Update.newBuilder()
+                                     .setEntity(Entity.newBuilder()
+                                                        .setActionProfileMember(actionProfileMember)
+                                                        .build())
+                                     .setType(UPDATE_TYPES.get(opType))
+                                     .build())
+                .collect(Collectors.toList());
+
+        if (updateMsgs.size() == 0) {
+            // Nothing to update
+            return true;
+        }
+
+        writeRequestBuilder
+                .setDeviceId(p4DeviceId)
+                .addAllUpdates(updateMsgs);
+        try {
+            blockingStub.write(writeRequestBuilder.build());
+            return true;
+        } catch (StatusRuntimeException e) {
+            log.warn("Unable to write table entries ({}): {}", opType, e.getMessage());
+            return false;
+        }
+    }
+
+    private Collection<PiActionGroup> doDumpGroups(PiActionProfileId piActionProfileId, PiPipeconf pipeconf) {
+        log.debug("Dumping groups from action profile {} from {} (pipeconf {})...",
+                  piActionProfileId.id(), deviceId, pipeconf.id());
+        P4InfoBrowser browser = PipeconfHelper.getP4InfoBrowser(pipeconf);
+        if (browser == null) {
+            log.warn("Unable to get a P4Info browser for pipeconf {}, skipping dump action profile {}",
+                     pipeconf, piActionProfileId);
+            return Collections.emptySet();
+        }
+
+        int actionProfileId;
+        try {
+            P4InfoOuterClass.ActionProfile actionProfile =
+                    browser.actionProfiles().getByName(piActionProfileId.id());
+            actionProfileId = actionProfile.getPreamble().getId();
+        } catch (P4InfoBrowser.NotFoundException e) {
+            log.warn("Can't find action profile {} from p4info", piActionProfileId);
+            return Collections.emptySet();
+        }
+
+        ActionProfileGroup actionProfileGroup =
+                ActionProfileGroup.newBuilder()
+                        .setActionProfileId(actionProfileId)
+                        .build();
+
+        ReadRequest requestMsg = ReadRequest.newBuilder()
+                .setDeviceId(p4DeviceId)
+                .addEntities(Entity.newBuilder()
+                                     .setActionProfileGroup(actionProfileGroup)
+                                     .build())
+                .build();
+
+        Iterator<ReadResponse> responses;
+        try {
+            responses = blockingStub.read(requestMsg);
+        } catch (StatusRuntimeException e) {
+            log.warn("Unable to read action profile {} due to {}", piActionProfileId, e.getMessage());
+            return Collections.emptySet();
+        }
+
+        List<ActionProfileGroup> actionProfileGroups =
+                Tools.stream(() -> responses)
+                        .map(ReadResponse::getEntitiesList)
+                        .flatMap(List::stream)
+                        .filter(entity -> entity.getEntityCase() == ACTION_PROFILE_GROUP)
+                        .map(Entity::getActionProfileGroup)
+                        .collect(Collectors.toList());
+
+        log.debug("Retrieved {} groups from action profile {} on {}...",
+                  actionProfileGroups.size(), piActionProfileId.id(), deviceId);
+
+        // group id -> members
+        Multimap<Integer, ActionProfileMember> actionProfileMemberMap = HashMultimap.create();
+        AtomicLong memberCount = new AtomicLong(0);
+        AtomicBoolean success = new AtomicBoolean(true);
+        actionProfileGroups.forEach(actProfGrp -> {
+            actProfGrp.getMembersList().forEach(member -> {
+                ActionProfileMember actProfMember =
+                        ActionProfileMember.newBuilder()
+                                .setActionProfileId(actProfGrp.getActionProfileId())
+                                .setMemberId(member.getMemberId())
+                                .build();
+                Entity entity = Entity.newBuilder()
+                        .setActionProfileMember(actProfMember)
+                        .build();
+
+                ReadRequest reqMsg = ReadRequest.newBuilder().setDeviceId(p4DeviceId)
+                        .addEntities(entity)
+                        .build();
+
+                Iterator<ReadResponse> resps;
+                try {
+                    resps = blockingStub.read(reqMsg);
+                } catch (StatusRuntimeException e) {
+                    log.warn("Unable to read member {} from action profile {} due to {}",
+                             member, piActionProfileId, e.getMessage());
+                    success.set(false);
+                    return;
+                }
+                Tools.stream(() -> resps)
+                        .map(ReadResponse::getEntitiesList)
+                        .flatMap(List::stream)
+                        .filter(e -> e.getEntityCase() == ACTION_PROFILE_MEMBER)
+                        .map(Entity::getActionProfileMember)
+                        .forEach(m -> {
+                            actionProfileMemberMap.put(actProfGrp.getGroupId(), m);
+                            memberCount.incrementAndGet();
+                        });
+            });
+        });
+
+        if (!success.get()) {
+            // Can't read members
+            return Collections.emptySet();
+        }
+        log.info("Retrieved {} group members from action profile {} on {}...",
+                 memberCount.get(), piActionProfileId.id(), deviceId);
+
+        Collection<PiActionGroup> piActionGroups = Sets.newHashSet();
+
+        for (ActionProfileGroup apg : actionProfileGroups) {
+            try {
+                Collection<ActionProfileMember> members = actionProfileMemberMap.get(apg.getGroupId());
+                PiActionGroup decodedGroup =
+                        ActionProfileGroupEncoder.decode(apg, members, pipeconf);
+                piActionGroups.add(decodedGroup);
+            } catch (EncodeException | P4InfoBrowser.NotFoundException e) {
+                log.warn("Can't decode group {} due to {}", apg, e.getMessage());
+                return Collections.emptySet();
+            }
+        }
+
+        return piActionGroups;
+    }
+
+    private boolean doWriteActionGroup(PiActionGroup group, WriteOperationType opType, PiPipeconf pipeconf) {
+        WriteRequest.Builder writeRequestBuilder = WriteRequest.newBuilder();
+        ActionProfileGroup actionProfileGroup;
+        try {
+            actionProfileGroup = ActionProfileGroupEncoder.encode(group, pipeconf);
+        } catch (EncodeException | P4InfoBrowser.NotFoundException e) {
+            log.warn("Can't encode group {} due to {}", e.getMessage());
+            return false;
+        }
+        Update updateMessage = Update.newBuilder()
+                .setEntity(Entity.newBuilder()
+                                   .setActionProfileGroup(actionProfileGroup)
+                                   .build())
+                .setType(UPDATE_TYPES.get(opType))
+                .build();
+        writeRequestBuilder
+                .setDeviceId(p4DeviceId)
+                .addUpdates(updateMessage);
+        try {
+            blockingStub.write(writeRequestBuilder.build());
+            return true;
+        } catch (StatusRuntimeException e) {
+            log.warn("Unable to write table entries ({}): {}", opType, e.getMessage());
+            return false;
+        }
+    }
+
     /**
      * Returns the internal P4 device ID associated with this client.
      *
diff --git a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeControllerImpl.java b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeControllerImpl.java
index e26341b..383b857 100644
--- a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeControllerImpl.java
+++ b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeControllerImpl.java
@@ -186,7 +186,7 @@
         }
     }
 
-    void postEvent(P4RuntimeEvent event) {
+    public void postEvent(P4RuntimeEvent event) {
         post(event);
     }
 }
diff --git a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeUtils.java b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeUtils.java
new file mode 100644
index 0000000..411e1fb
--- /dev/null
+++ b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/P4RuntimeUtils.java
@@ -0,0 +1,51 @@
+/*
+ * 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.ctl;
+
+import com.google.protobuf.ByteString;
+
+import static java.lang.String.format;
+
+/**
+ * Utilities for P4 runtime control.
+ */
+public final class P4RuntimeUtils {
+
+    private P4RuntimeUtils() {
+        // Hide default construction
+    }
+
+    static void assertSize(String entityDescr, ByteString value, int bitWidth)
+            throws EncodeException {
+
+        int byteWidth = (int) Math.ceil((float) bitWidth / 8);
+        if (value.size() != byteWidth) {
+            throw new EncodeException(format("Wrong size for %s, expected %d bytes, but found %d",
+                                             entityDescr, byteWidth, value.size()));
+        }
+    }
+
+    static void assertPrefixLen(String entityDescr, int prefixLength, int bitWidth)
+            throws EncodeException {
+
+        if (prefixLength > bitWidth) {
+            throw new EncodeException(format(
+                    "wrong prefix length for %s, field size is %d bits, but found one is %d",
+                    entityDescr, bitWidth, prefixLength));
+        }
+    }
+}
diff --git a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/TableEntryEncoder.java b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/TableEntryEncoder.java
index 9ffc18f..8278b3e 100644
--- a/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/TableEntryEncoder.java
+++ b/protocols/p4runtime/ctl/src/main/java/org/onosproject/p4runtime/ctl/TableEntryEncoder.java
@@ -17,10 +17,13 @@
 package org.onosproject.p4runtime.ctl;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import org.onlab.util.ImmutableByteSequence;
 import org.onosproject.net.pi.model.PiPipeconf;
 import org.onosproject.net.pi.runtime.PiAction;
+import org.onosproject.net.pi.runtime.PiActionGroupId;
+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;
@@ -36,25 +39,26 @@
 import org.onosproject.net.pi.runtime.PiTernaryFieldMatch;
 import org.onosproject.net.pi.runtime.PiValidFieldMatch;
 import org.slf4j.Logger;
-import p4.P4RuntimeOuterClass.Action;
 import p4.P4RuntimeOuterClass.FieldMatch;
 import p4.P4RuntimeOuterClass.TableAction;
 import p4.P4RuntimeOuterClass.TableEntry;
+import p4.P4RuntimeOuterClass.Action;
 import p4.config.P4InfoOuterClass;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 
 import static java.lang.String.format;
 import static org.onlab.util.ImmutableByteSequence.copyFrom;
+import static org.onosproject.p4runtime.ctl.P4RuntimeUtils.assertPrefixLen;
+import static org.onosproject.p4runtime.ctl.P4RuntimeUtils.assertSize;
 import static org.slf4j.LoggerFactory.getLogger;
 
 /**
- * Encoder of table entries, from ONOS Pi* format, to P4Runtime protobuf messages, and vice versa.
+ * Encoder/Decoder of table entries, from ONOS Pi* format, to P4Runtime protobuf messages, and vice versa.
  */
 final class TableEntryEncoder {
-
-
     private static final Logger log = getLogger(TableEntryEncoder.class);
 
     private static final String HEADER_PREFIX = "hdr.";
@@ -422,7 +426,7 @@
         }
     }
 
-    private static TableAction encodePiTableAction(PiTableAction piTableAction, P4InfoBrowser browser)
+    static TableAction encodePiTableAction(PiTableAction piTableAction, P4InfoBrowser browser)
             throws P4InfoBrowser.NotFoundException, EncodeException {
 
         TableAction.Builder tableActionMsgBuilder = TableAction.newBuilder();
@@ -430,24 +434,17 @@
         switch (piTableAction.type()) {
             case ACTION:
                 PiAction piAction = (PiAction) piTableAction;
-                int actionId = browser.actions().getByName(piAction.id().name()).getPreamble().getId();
-
-                Action.Builder actionMsgBuilder = Action.newBuilder().setActionId(actionId);
-
-                for (PiActionParam p : piAction.parameters()) {
-                    P4InfoOuterClass.Action.Param paramInfo = browser.actionParams(actionId).getByName(p.id().name());
-                    ByteString paramValue = ByteString.copyFrom(p.value().asReadOnlyBuffer());
-                    assertSize(format("param '%s' of action '%s'", p.id(), piAction.id()),
-                               paramValue, paramInfo.getBitwidth());
-                    actionMsgBuilder.addParams(Action.Param.newBuilder()
-                                                       .setParamId(paramInfo.getId())
-                                                       .setValue(paramValue)
-                                                       .build());
-                }
-
-                tableActionMsgBuilder.setAction(actionMsgBuilder.build());
+                Action theAction = encodePiAction(piAction, browser);
+                tableActionMsgBuilder.setAction(theAction);
                 break;
-
+            case ACTION_GROUP_ID:
+                PiActionGroupId actionGroupId = (PiActionGroupId) piTableAction;
+                tableActionMsgBuilder.setActionProfileGroupId(actionGroupId.id());
+                break;
+            case GROUP_MEMBER_ID:
+                PiActionGroupMemberId actionGroupMemberId = (PiActionGroupMemberId) piTableAction;
+                tableActionMsgBuilder.setActionProfileMemberId(actionGroupMemberId.id());
+                break;
             default:
                 throw new EncodeException(
                         format("Building of table action type %s not implemented", piTableAction.type()));
@@ -456,50 +453,56 @@
         return tableActionMsgBuilder.build();
     }
 
-    private static PiTableAction decodeTableActionMsg(TableAction tableActionMsg, P4InfoBrowser browser)
+    static PiTableAction decodeTableActionMsg(TableAction tableActionMsg, P4InfoBrowser browser)
             throws P4InfoBrowser.NotFoundException, EncodeException {
-
         TableAction.TypeCase typeCase = tableActionMsg.getTypeCase();
-
         switch (typeCase) {
             case ACTION:
-                PiAction.Builder piActionBuilder = PiAction.builder();
                 Action actionMsg = tableActionMsg.getAction();
-                // Action ID.
-                int actionId = actionMsg.getActionId();
-                String actionName = browser.actions().getById(actionId).getPreamble().getName();
-                piActionBuilder.withId(PiActionId.of(actionName));
-                // Params.
-                for (Action.Param paramMsg : actionMsg.getParamsList()) {
-                    String paramName = browser.actionParams(actionId).getById(paramMsg.getParamId()).getName();
-                    ImmutableByteSequence paramValue = copyFrom(paramMsg.getValue().asReadOnlyByteBuffer());
-                    piActionBuilder.withParameter(new PiActionParam(PiActionParamId.of(paramName), paramValue));
-                }
-                return piActionBuilder.build();
-
+                return decodeActionMsg(actionMsg, browser);
             default:
                 throw new EncodeException(
                         format("Decoding of table action type %s not implemented", typeCase.name()));
         }
     }
 
-    private static void assertSize(String entityDescr, ByteString value, int bitWidth)
-            throws EncodeException {
+    static Action encodePiAction(PiAction piAction, P4InfoBrowser browser)
+            throws P4InfoBrowser.NotFoundException, EncodeException {
 
-        int byteWidth = (int) Math.ceil((float) bitWidth / 8);
-        if (value.size() != byteWidth) {
-            throw new EncodeException(format("Wrong size for %s, expected %d bytes, but found %d",
-                                             entityDescr, byteWidth, value.size()));
+        int actionId = browser.actions().getByName(piAction.id().name()).getPreamble().getId();
+
+        Action.Builder actionMsgBuilder =
+                Action.newBuilder().setActionId(actionId);
+
+        for (PiActionParam p : piAction.parameters()) {
+            P4InfoOuterClass.Action.Param paramInfo = browser.actionParams(actionId).getByName(p.id().name());
+            ByteString paramValue = ByteString.copyFrom(p.value().asReadOnlyBuffer());
+            assertSize(format("param '%s' of action '%s'", p.id(), piAction.id()),
+                       paramValue, paramInfo.getBitwidth());
+            actionMsgBuilder.addParams(Action.Param.newBuilder()
+                                               .setParamId(paramInfo.getId())
+                                               .setValue(paramValue)
+                                               .build());
         }
+
+        return actionMsgBuilder.build();
     }
 
-    private static void assertPrefixLen(String entityDescr, int prefixLength, int bitWidth)
-            throws EncodeException {
+    static PiAction decodeActionMsg(Action action, P4InfoBrowser browser)
+            throws P4InfoBrowser.NotFoundException {
+        P4InfoBrowser.EntityBrowser<P4InfoOuterClass.Action.Param> paramInfo =
+                browser.actionParams(action.getActionId());
+        String actionName = browser.actions()
+                .getById(action.getActionId())
+                .getPreamble().getName();
+        PiActionId id = PiActionId.of(actionName);
+        List<PiActionParam> params = Lists.newArrayList();
 
-        if (prefixLength > bitWidth) {
-            throw new EncodeException(format(
-                    "wrong prefix length for %s, field size is %d bits, but found one is %d",
-                    entityDescr, bitWidth, prefixLength));
+        for (Action.Param p : action.getParamsList()) {
+            String paramName = paramInfo.getById(p.getParamId()).getName();
+            ImmutableByteSequence value = ImmutableByteSequence.copyFrom(p.getValue().toByteArray());
+            params.add(new PiActionParam(PiActionParamId.of(paramName), value));
         }
+        return PiAction.builder().withId(id).withParameters(params).build();
     }
 }
diff --git a/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/MockP4RuntimeServer.java b/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/MockP4RuntimeServer.java
new file mode 100644
index 0000000..4773458
--- /dev/null
+++ b/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/MockP4RuntimeServer.java
@@ -0,0 +1,114 @@
+/*
+ * 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.ctl;
+
+import com.google.common.collect.Lists;
+import io.grpc.stub.StreamObserver;
+import p4.P4RuntimeGrpc;
+import p4.P4RuntimeOuterClass;
+import p4.P4RuntimeOuterClass.GetForwardingPipelineConfigRequest;
+import p4.P4RuntimeOuterClass.GetForwardingPipelineConfigResponse;
+import p4.P4RuntimeOuterClass.ReadRequest;
+import p4.P4RuntimeOuterClass.ReadResponse;
+import p4.P4RuntimeOuterClass.SetForwardingPipelineConfigRequest;
+import p4.P4RuntimeOuterClass.SetForwardingPipelineConfigResponse;
+import p4.P4RuntimeOuterClass.StreamMessageResponse;
+import p4.P4RuntimeOuterClass.WriteRequest;
+import p4.P4RuntimeOuterClass.WriteResponse;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class MockP4RuntimeServer extends P4RuntimeGrpc.P4RuntimeImplBase {
+    private CompletableFuture<Void> completeLock;
+    private AtomicLong counter;
+
+    // Requests
+    private List<WriteRequest> writeReqs;
+    private List<ReadRequest> readReqs;
+    private List<ReadResponse> readResps;
+
+    /**
+     * Expect N times request sent by client.
+     *
+     * @param times the number of request will sent by client.
+     * @return a completable future object, wll complete after client send N times requests.
+     */
+    public CompletableFuture<Void> expectRequests(long times) {
+        counter = new AtomicLong(times);
+        completeLock = new CompletableFuture<>();
+        readReqs = Lists.newArrayList();
+        writeReqs = Lists.newArrayList();
+        return completeLock;
+    }
+
+    private void complete() {
+        if (counter.decrementAndGet() == 0) {
+            completeLock.complete(null);
+        }
+    }
+
+    public void willReturnReadResult(Collection<ReadResponse> readResps) {
+        this.readResps = Lists.newArrayList(readResps);
+    }
+
+    public List<WriteRequest> getWriteReqs() {
+        return writeReqs;
+    }
+
+    public List<ReadRequest> getReadReqs() {
+        return readReqs;
+    }
+
+    @Override
+    public void write(WriteRequest request, StreamObserver<WriteResponse> responseObserver) {
+        writeReqs.add(request);
+        complete();
+    }
+
+    @Override
+    public void read(ReadRequest request, StreamObserver<ReadResponse> responseObserver) {
+        readReqs.add(request);
+        if (readResps != null && !readResps.isEmpty()) {
+            ReadResponse readResp = readResps.remove(0); // get first response
+            responseObserver.onNext(readResp);
+            responseObserver.onCompleted();
+        }
+        complete();
+    }
+
+    @Override
+    public void setForwardingPipelineConfig(SetForwardingPipelineConfigRequest request,
+                                            StreamObserver<SetForwardingPipelineConfigResponse> responseObserver) {
+        throw new UnsupportedOperationException("Not implement yet");
+    }
+
+    @Override
+    public void getForwardingPipelineConfig(GetForwardingPipelineConfigRequest request,
+                                            StreamObserver<GetForwardingPipelineConfigResponse> responseObserver) {
+        throw new UnsupportedOperationException("Not implement yet");
+    }
+
+    @Override
+    public StreamObserver<P4RuntimeOuterClass.StreamMessageRequest>
+    streamChannel(StreamObserver<StreamMessageResponse> responseObserver) {
+        // TODO: not implement yet
+        return null;
+    }
+}
diff --git a/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/P4RuntimeGroupTest.java b/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/P4RuntimeGroupTest.java
new file mode 100644
index 0000000..8932a16
--- /dev/null
+++ b/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/P4RuntimeGroupTest.java
@@ -0,0 +1,279 @@
+/*
+ * 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.ctl;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import io.grpc.ManagedChannel;
+import io.grpc.Server;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import io.grpc.inprocess.InProcessServerBuilder;
+import io.grpc.internal.AbstractServerImplBuilder;
+import org.easymock.EasyMock;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.onlab.util.ImmutableByteSequence;
+import org.onosproject.net.DeviceId;
+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.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.PiActionGroupId;
+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 p4.P4RuntimeOuterClass.ActionProfileGroup;
+import p4.P4RuntimeOuterClass.ActionProfileMember;
+import p4.P4RuntimeOuterClass.Entity;
+import p4.P4RuntimeOuterClass.Update;
+import p4.P4RuntimeOuterClass.WriteRequest;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.easymock.EasyMock.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.onosproject.net.pi.model.PiPipeconf.ExtensionType.P4_INFO_TEXT;
+import static org.onosproject.net.pi.runtime.PiActionGroup.Type.SELECT;
+import static org.onosproject.p4runtime.api.P4RuntimeClient.WriteOperationType.INSERT;
+import static p4.P4RuntimeOuterClass.*;
+
+/**
+ * Tests for P4 Runtime Action Profile Group support.
+ */
+public class P4RuntimeGroupTest {
+    private static final String PIPECONF_ID = "p4runtime-mock-pipeconf";
+    private static final String P4INFO_PATH = "/default.p4info";
+    private static final PiPipeconf PIPECONF = buildPipeconf();
+    private static final int P4_INFO_ACT_PROF_ID = 285227860;
+    private static final PiActionProfileId ACT_PROF_ID = PiActionProfileId.of("ecmp_selector");
+    private static final PiActionGroupId GROUP_ID = PiActionGroupId.of(1);
+    private static final int DEFAULT_MEMBER_WEIGHT = 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 int BASE_MEM_ID = 65535;
+    private static final List<Integer> MEMBER_IDS = ImmutableList.of(65536, 65537, 65538);
+    private static final Collection<PiActionGroupMember> GROUP_MEMBERS =
+            ImmutableSet.of(
+                    outputMember((short) 1),
+                    outputMember((short) 2),
+                    outputMember((short) 3)
+            );
+    private static final PiActionGroup GROUP = PiActionGroup.builder()
+            .withId(GROUP_ID)
+            .addMembers(GROUP_MEMBERS)
+            .withActionProfileId(ACT_PROF_ID)
+            .withType(SELECT)
+            .build();
+    private static final DeviceId DEVICE_ID = DeviceId.deviceId("device:p4runtime:1");
+    private static final int P4_DEVICE_ID = 1;
+    private static final int SET_EGRESS_PORT_ID = 16794308;
+    private static final String GRPC_SERVER_NAME = "P4RuntimeGroupTest";
+    private static final long DEFAULT_TIMEOUT_TIME = 5;
+
+    private P4RuntimeClientImpl client;
+    private P4RuntimeControllerImpl controller;
+    private static MockP4RuntimeServer p4RuntimeServerImpl = new MockP4RuntimeServer();
+    private static Server grpcServer;
+    private static ManagedChannel grpcChannel;
+
+    private static PiActionGroupMember outputMember(short 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();
+    }
+
+    private static PiPipeconf buildPipeconf() {
+        final URL p4InfoUrl = P4RuntimeGroupTest.class.getResource(P4INFO_PATH);
+        return DefaultPiPipeconf.builder()
+                .withId(new PiPipeconfId(PIPECONF_ID))
+                .withPipelineModel(EasyMock.niceMock(PiPipelineModel.class))
+                .addExtension(P4_INFO_TEXT, p4InfoUrl)
+                .build();
+    }
+
+    @BeforeClass
+    public static void globalSetup() throws IOException {
+        AbstractServerImplBuilder builder = InProcessServerBuilder
+                .forName(GRPC_SERVER_NAME).directExecutor();
+        builder.addService(p4RuntimeServerImpl);
+        grpcServer = builder.build().start();
+        grpcChannel = InProcessChannelBuilder.forName(GRPC_SERVER_NAME)
+                .directExecutor()
+                .usePlaintext(true)
+                .build();
+    }
+
+    @AfterClass
+    public static void globalTeerDown() {
+        grpcServer.shutdown();
+        grpcChannel.shutdown();
+    }
+
+
+    @Before
+    public void setup() {
+        controller = niceMock(P4RuntimeControllerImpl.class);
+        client = new P4RuntimeClientImpl(DEVICE_ID, P4_DEVICE_ID,
+                                         grpcChannel,
+                                         controller);
+    }
+
+    @Test
+    public void testInsertPiActionGroup() throws Exception {
+        CompletableFuture<Void> complete = p4RuntimeServerImpl.expectRequests(1);
+        client.writeActionGroup(GROUP, INSERT, PIPECONF);
+        complete.get(DEFAULT_TIMEOUT_TIME, TimeUnit.SECONDS);
+        WriteRequest result = p4RuntimeServerImpl.getWriteReqs().get(0);
+        assertEquals(1, result.getDeviceId());
+        assertEquals(1, result.getUpdatesCount());
+
+        Update update = result.getUpdatesList().get(0);
+        assertEquals(Update.Type.INSERT, update.getType());
+
+        Entity entity = update.getEntity();
+        ActionProfileGroup actionProfileGroup = entity.getActionProfileGroup();
+        assertNotNull(actionProfileGroup);
+
+        assertEquals(P4_INFO_ACT_PROF_ID, actionProfileGroup.getActionProfileId());
+        assertEquals(3, actionProfileGroup.getMembersCount());
+        List<ActionProfileGroup.Member> members = actionProfileGroup.getMembersList();
+
+        for (ActionProfileGroup.Member member : members) {
+            // XXX: We can't guarantee the order of member, just make sure we
+            // have these member ids
+            assertTrue(MEMBER_IDS.contains(member.getMemberId()));
+            assertEquals(DEFAULT_MEMBER_WEIGHT, member.getWeight());
+        }
+    }
+
+    @Test
+    public void testInsertPiActionMembers() throws Exception {
+        CompletableFuture<Void> complete = p4RuntimeServerImpl.expectRequests(1);
+        client.writeActionGroupMembers(GROUP, GROUP_MEMBERS, INSERT, PIPECONF);
+        complete.get(DEFAULT_TIMEOUT_TIME, TimeUnit.SECONDS);
+        WriteRequest result = p4RuntimeServerImpl.getWriteReqs().get(0);
+        assertEquals(1, result.getDeviceId());
+        assertEquals(3, result.getUpdatesCount());
+
+        List<Update> updates = result.getUpdatesList();
+        for (Update update : updates) {
+            assertEquals(Update.Type.INSERT, update.getType());
+            Entity entity = update.getEntity();
+            ActionProfileMember member = entity.getActionProfileMember();
+            assertNotNull(member);
+            assertEquals(P4_INFO_ACT_PROF_ID, member.getActionProfileId());
+            assertTrue(MEMBER_IDS.contains(member.getMemberId()));
+            Action action = member.getAction();
+            assertEquals(SET_EGRESS_PORT_ID, action.getActionId());
+            assertEquals(1, action.getParamsCount());
+            Action.Param param = action.getParamsList().get(0);
+            assertEquals(1, param.getParamId());
+            byte outPort = (byte) (member.getMemberId() - BASE_MEM_ID);
+            ByteString bs = ByteString.copyFrom(new byte[]{0, outPort});
+            assertEquals(bs, param.getValue());
+        }
+    }
+
+    @Test
+    public void testReadGroups() throws Exception {
+        ActionProfileGroup.Builder group = ActionProfileGroup.newBuilder()
+                .setGroupId(GROUP_ID.id())
+                .setType(ActionProfileGroup.Type.SELECT)
+                .setActionProfileId(P4_INFO_ACT_PROF_ID);
+
+        List<ActionProfileMember> members = Lists.newArrayList();
+
+        MEMBER_IDS.forEach(id -> {
+            ActionProfileGroup.Member member = ActionProfileGroup.Member.newBuilder()
+                    .setMemberId(id)
+                    .setWeight(DEFAULT_MEMBER_WEIGHT)
+                    .build();
+            group.addMembers(member);
+
+            byte outPort = (byte) (id - BASE_MEM_ID);
+            ByteString bs = ByteString.copyFrom(new byte[]{0, outPort});
+            Action.Param param = Action.Param.newBuilder()
+                    .setParamId(1)
+                    .setValue(bs)
+                    .build();
+
+            Action action = Action.newBuilder()
+                    .setActionId(SET_EGRESS_PORT_ID)
+                    .addParams(param)
+                    .build();
+
+
+            ActionProfileMember actProfMember =
+                    ActionProfileMember.newBuilder()
+                            .setMemberId(id)
+                            .setAction(action)
+                            .build();
+            members.add(actProfMember);
+        });
+
+        List<ReadResponse> responses = Lists.newArrayList();
+        responses.add(ReadResponse.newBuilder()
+                              .addEntities(Entity.newBuilder().setActionProfileGroup(group))
+                              .build()
+        );
+
+        members.forEach(m -> {
+            responses.add(ReadResponse.newBuilder()
+                                  .addEntities(Entity.newBuilder().setActionProfileMember(m))
+                                  .build());
+        });
+
+        p4RuntimeServerImpl.willReturnReadResult(responses);
+        CompletableFuture<Void> complete = p4RuntimeServerImpl.expectRequests(4);
+        CompletableFuture<Collection<PiActionGroup>> groupsComplete = client.dumpGroups(ACT_PROF_ID, PIPECONF);
+        complete.get(DEFAULT_TIMEOUT_TIME, TimeUnit.SECONDS);
+
+        Collection<PiActionGroup> groups = groupsComplete.get(DEFAULT_TIMEOUT_TIME, TimeUnit.SECONDS);
+        assertEquals(1, groups.size());
+        PiActionGroup piActionGroup = groups.iterator().next();
+        assertEquals(ACT_PROF_ID, piActionGroup.actionProfileId());
+        assertEquals(GROUP_ID, piActionGroup.id());
+        assertEquals(SELECT, piActionGroup.type());
+        assertEquals(3, piActionGroup.members().size());
+        assertTrue(GROUP_MEMBERS.containsAll(piActionGroup.members()));
+        assertTrue(piActionGroup.members().containsAll(GROUP_MEMBERS));
+    }
+}
diff --git a/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/TableEntryEncoderTest.java b/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/TableEntryEncoderTest.java
index 792e838..b3e9a1e 100644
--- a/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/TableEntryEncoderTest.java
+++ b/protocols/p4runtime/ctl/src/test/java/org/onosproject/p4runtime/ctl/TableEntryEncoderTest.java
@@ -18,12 +18,13 @@
 
 import com.google.common.collect.Lists;
 import com.google.common.testing.EqualsTester;
+import org.easymock.EasyMock;
 import org.junit.Test;
 import org.onlab.util.ImmutableByteSequence;
-import org.onosproject.bmv2.model.Bmv2PipelineModelParser;
 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.PiPipelineModel;
 import org.onosproject.net.pi.runtime.PiAction;
 import org.onosproject.net.pi.runtime.PiActionId;
 import org.onosproject.net.pi.runtime.PiActionParam;
@@ -33,7 +34,6 @@
 import org.onosproject.net.pi.runtime.PiTableEntry;
 import org.onosproject.net.pi.runtime.PiTableId;
 import org.onosproject.net.pi.runtime.PiTernaryFieldMatch;
-import org.slf4j.Logger;
 import p4.P4RuntimeOuterClass.Action;
 import p4.P4RuntimeOuterClass.TableEntry;
 
@@ -47,19 +47,14 @@
 import static org.onlab.util.ImmutableByteSequence.copyFrom;
 import static org.onlab.util.ImmutableByteSequence.fit;
 import static org.onlab.util.ImmutableByteSequence.ofOnes;
-import static org.onosproject.net.pi.model.PiPipeconf.ExtensionType.BMV2_JSON;
 import static org.onosproject.net.pi.model.PiPipeconf.ExtensionType.P4_INFO_TEXT;
 import static org.onosproject.p4runtime.ctl.TableEntryEncoder.decode;
 import static org.onosproject.p4runtime.ctl.TableEntryEncoder.encode;
-import static org.slf4j.LoggerFactory.getLogger;
 
-//import org.onosproject.driver.pipeline.DefaultSingleTablePipeline;
-//import org.onosproject.drivers.bmv2.Bmv2DefaultInterpreter;
-
+/**
+ * Test for P4 runtime table entry encoder.
+ */
 public class TableEntryEncoderTest {
-
-    private final Logger log = getLogger(getClass());
-
     private static final String TABLE_0 = "table0";
     private static final String SET_EGRESS_PORT = "set_egress_port";
     private static final String PORT = "port";
@@ -72,14 +67,11 @@
 
     private final Random rand = new Random();
     private final URL p4InfoUrl = this.getClass().getResource("/default.p4info");
-    private final URL jsonUrl = this.getClass().getResource("/default.json");
 
     private final PiPipeconf defaultPipeconf = DefaultPiPipeconf.builder()
             .withId(new PiPipeconfId("mock"))
-            .withPipelineModel(Bmv2PipelineModelParser.parse(jsonUrl))
-//            .addBehaviour(PiPipelineInterpreter.class, Bmv2DefaultInterpreter.class)
+            .withPipelineModel(EasyMock.niceMock(PiPipelineModel.class))
             .addExtension(P4_INFO_TEXT, p4InfoUrl)
-            .addExtension(BMV2_JSON, jsonUrl)
             .build();
 
     private final P4InfoBrowser browser = PipeconfHelper.getP4InfoBrowser(defaultPipeconf);
diff --git a/protocols/p4runtime/ctl/src/test/resources/default.json b/protocols/p4runtime/ctl/src/test/resources/default.json
deleted file mode 100644
index 3a46dcc..0000000
--- a/protocols/p4runtime/ctl/src/test/resources/default.json
+++ /dev/null
@@ -1,2697 +0,0 @@
-{
-  "program" : "default.p4",
-  "__meta__" : {
-    "version" : [2, 7],
-    "compiler" : "https://github.com/p4lang/p4c"
-  },
-  "header_types" : [
-    {
-      "name" : "scalars_0",
-      "id" : 0,
-      "fields" : [
-        ["tmp", 32, false],
-        ["tmp_0", 32, false]
-      ]
-    },
-    {
-      "name" : "standard_metadata",
-      "id" : 1,
-      "fields" : [
-        ["ingress_port", 9, false],
-        ["egress_spec", 9, false],
-        ["egress_port", 9, false],
-        ["clone_spec", 32, false],
-        ["instance_type", 32, false],
-        ["drop", 1, false],
-        ["recirculate_port", 16, false],
-        ["packet_length", 32, false],
-        ["enq_timestamp", 32, false],
-        ["enq_qdepth", 19, false],
-        ["deq_timedelta", 32, false],
-        ["deq_qdepth", 19, false],
-        ["ingress_global_timestamp", 48, false],
-        ["lf_field_list", 32, false],
-        ["mcast_grp", 16, false],
-        ["resubmit_flag", 1, false],
-        ["egress_rid", 16, false],
-        ["_padding", 5, false]
-      ]
-    },
-    {
-      "name" : "ethernet_t",
-      "id" : 2,
-      "fields" : [
-        ["dstAddr", 48, false],
-        ["srcAddr", 48, false],
-        ["etherType", 16, false]
-      ]
-    },
-    {
-      "name" : "ipv4_t",
-      "id" : 3,
-      "fields" : [
-        ["version", 4, false],
-        ["ihl", 4, false],
-        ["diffserv", 8, false],
-        ["totalLen", 16, false],
-        ["identification", 16, false],
-        ["flags", 3, false],
-        ["fragOffset", 13, false],
-        ["ttl", 8, false],
-        ["protocol", 8, false],
-        ["hdrChecksum", 16, false],
-        ["srcAddr", 32, false],
-        ["dstAddr", 32, false]
-      ]
-    },
-    {
-      "name" : "tcp_t",
-      "id" : 4,
-      "fields" : [
-        ["srcPort", 16, false],
-        ["dstPort", 16, false],
-        ["seqNo", 32, false],
-        ["ackNo", 32, false],
-        ["dataOffset", 4, false],
-        ["res", 3, false],
-        ["ecn", 3, false],
-        ["ctrl", 6, false],
-        ["window", 16, false],
-        ["checksum", 16, false],
-        ["urgentPtr", 16, false]
-      ]
-    },
-    {
-      "name" : "udp_t",
-      "id" : 5,
-      "fields" : [
-        ["srcPort", 16, false],
-        ["dstPort", 16, false],
-        ["length_", 16, false],
-        ["checksum", 16, false]
-      ]
-    },
-    {
-      "name" : "ecmp_metadata_t",
-      "id" : 6,
-      "fields" : [
-        ["groupId", 16, false],
-        ["selector", 16, false]
-      ]
-    },
-    {
-      "name" : "wcmp_meta_t",
-      "id" : 7,
-      "fields" : [
-        ["groupId", 16, false],
-        ["numBits", 8, false],
-        ["selector", 64, false]
-      ]
-    },
-    {
-      "name" : "intrinsic_metadata_t",
-      "id" : 8,
-      "fields" : [
-        ["ingress_global_timestamp", 32, false],
-        ["lf_field_list", 32, false],
-        ["mcast_grp", 16, false],
-        ["egress_rid", 16, false]
-      ]
-    }
-  ],
-  "headers" : [
-    {
-      "name" : "standard_metadata_3",
-      "id" : 0,
-      "header_type" : "standard_metadata",
-      "metadata" : true,
-      "pi_omit" : true
-    },
-    {
-      "name" : "standard_metadata_4",
-      "id" : 1,
-      "header_type" : "standard_metadata",
-      "metadata" : true,
-      "pi_omit" : true
-    },
-    {
-      "name" : "standard_metadata_5",
-      "id" : 2,
-      "header_type" : "standard_metadata",
-      "metadata" : true,
-      "pi_omit" : true
-    },
-    {
-      "name" : "scalars",
-      "id" : 3,
-      "header_type" : "scalars_0",
-      "metadata" : true,
-      "pi_omit" : true
-    },
-    {
-      "name" : "standard_metadata",
-      "id" : 4,
-      "header_type" : "standard_metadata",
-      "metadata" : true,
-      "pi_omit" : true
-    },
-    {
-      "name" : "ethernet",
-      "id" : 5,
-      "header_type" : "ethernet_t",
-      "metadata" : false,
-      "pi_omit" : true
-    },
-    {
-      "name" : "ipv4",
-      "id" : 6,
-      "header_type" : "ipv4_t",
-      "metadata" : false,
-      "pi_omit" : true
-    },
-    {
-      "name" : "tcp",
-      "id" : 7,
-      "header_type" : "tcp_t",
-      "metadata" : false,
-      "pi_omit" : true
-    },
-    {
-      "name" : "udp",
-      "id" : 8,
-      "header_type" : "udp_t",
-      "metadata" : false,
-      "pi_omit" : true
-    },
-    {
-      "name" : "ecmp_metadata",
-      "id" : 9,
-      "header_type" : "ecmp_metadata_t",
-      "metadata" : true,
-      "pi_omit" : true
-    },
-    {
-      "name" : "wcmp_meta",
-      "id" : 10,
-      "header_type" : "wcmp_meta_t",
-      "metadata" : true,
-      "pi_omit" : true
-    },
-    {
-      "name" : "intrinsic_metadata",
-      "id" : 11,
-      "header_type" : "intrinsic_metadata_t",
-      "metadata" : true,
-      "pi_omit" : true
-    }
-  ],
-  "header_stacks" : [],
-  "header_union_types" : [],
-  "header_unions" : [],
-  "header_union_stacks" : [],
-  "field_lists" : [],
-  "errors" : [
-    ["NoError", 1],
-    ["PacketTooShort", 2],
-    ["NoMatch", 3],
-    ["StackOutOfBounds", 4],
-    ["HeaderTooShort", 5],
-    ["ParserTimeout", 6]
-  ],
-  "enums" : [],
-  "parsers" : [
-    {
-      "name" : "parser",
-      "id" : 0,
-      "init_state" : "start",
-      "parse_states" : [
-        {
-          "name" : "parse_ipv4",
-          "id" : 0,
-          "parser_ops" : [
-            {
-              "parameters" : [
-                {
-                  "type" : "regular",
-                  "value" : "ipv4"
-                }
-              ],
-              "op" : "extract"
-            }
-          ],
-          "transitions" : [
-            {
-              "value" : "0x06",
-              "mask" : null,
-              "next_state" : "parse_tcp"
-            },
-            {
-              "value" : "0x11",
-              "mask" : null,
-              "next_state" : "parse_udp"
-            },
-            {
-              "value" : "default",
-              "mask" : null,
-              "next_state" : null
-            }
-          ],
-          "transition_key" : [
-            {
-              "type" : "field",
-              "value" : ["ipv4", "protocol"]
-            }
-          ]
-        },
-        {
-          "name" : "parse_tcp",
-          "id" : 1,
-          "parser_ops" : [
-            {
-              "parameters" : [
-                {
-                  "type" : "regular",
-                  "value" : "tcp"
-                }
-              ],
-              "op" : "extract"
-            }
-          ],
-          "transitions" : [
-            {
-              "value" : "default",
-              "mask" : null,
-              "next_state" : null
-            }
-          ],
-          "transition_key" : []
-        },
-        {
-          "name" : "parse_udp",
-          "id" : 2,
-          "parser_ops" : [
-            {
-              "parameters" : [
-                {
-                  "type" : "regular",
-                  "value" : "udp"
-                }
-              ],
-              "op" : "extract"
-            }
-          ],
-          "transitions" : [
-            {
-              "value" : "default",
-              "mask" : null,
-              "next_state" : null
-            }
-          ],
-          "transition_key" : []
-        },
-        {
-          "name" : "start",
-          "id" : 3,
-          "parser_ops" : [
-            {
-              "parameters" : [
-                {
-                  "type" : "regular",
-                  "value" : "ethernet"
-                }
-              ],
-              "op" : "extract"
-            }
-          ],
-          "transitions" : [
-            {
-              "value" : "0x0800",
-              "mask" : null,
-              "next_state" : "parse_ipv4"
-            },
-            {
-              "value" : "default",
-              "mask" : null,
-              "next_state" : null
-            }
-          ],
-          "transition_key" : [
-            {
-              "type" : "field",
-              "value" : ["ethernet", "etherType"]
-            }
-          ]
-        }
-      ]
-    }
-  ],
-  "deparsers" : [
-    {
-      "name" : "deparser",
-      "id" : 0,
-      "source_info" : {
-        "filename" : "include/parsers.p4",
-        "line" : 43,
-        "column" : 8,
-        "source_fragment" : "DeparserImpl"
-      },
-      "order" : ["ethernet", "ipv4", "udp", "tcp"]
-    }
-  ],
-  "meter_arrays" : [],
-  "counter_arrays" : [
-    {
-      "name" : "table0_counter",
-      "id" : 0,
-      "is_direct" : true,
-      "binding" : "table0"
-    },
-    {
-      "name" : "port_counters_control.egress_port_counter",
-      "id" : 1,
-      "source_info" : {
-        "filename" : "include/port_counters.p4",
-        "line" : 6,
-        "column" : 38,
-        "source_fragment" : "egress_port_counter"
-      },
-      "size" : 254,
-      "is_direct" : false
-    },
-    {
-      "name" : "port_counters_control.ingress_port_counter",
-      "id" : 2,
-      "source_info" : {
-        "filename" : "include/port_counters.p4",
-        "line" : 7,
-        "column" : 38,
-        "source_fragment" : "ingress_port_counter"
-      },
-      "size" : 254,
-      "is_direct" : false
-    }
-  ],
-  "register_arrays" : [],
-  "calculations" : [],
-  "learn_lists" : [],
-  "actions" : [
-    {
-      "name" : "set_egress_port",
-      "id" : 0,
-      "runtime_data" : [
-        {
-          "name" : "port",
-          "bitwidth" : 9
-        }
-      ],
-      "primitives" : [
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "ingress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "egress_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "egress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "clone_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "clone_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "instance_type"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "instance_type"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "drop"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "drop"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "recirculate_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "recirculate_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "packet_length"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "packet_length"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "enq_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "enq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "deq_timedelta"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_timedelta"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "deq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "ingress_global_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_global_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "lf_field_list"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "lf_field_list"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "mcast_grp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "mcast_grp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "resubmit_flag"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "resubmit_flag"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "egress_rid"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_rid"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "egress_spec"]
-            },
-            {
-              "type" : "runtime_data",
-              "value" : 0
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 9,
-            "column" : 4,
-            "source_fragment" : "standard_metadata.egress_spec = port"
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "ingress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "egress_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "egress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "clone_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "clone_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "instance_type"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "instance_type"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "drop"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "drop"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "recirculate_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "recirculate_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "packet_length"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "packet_length"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "enq_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "enq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_timedelta"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "deq_timedelta"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "deq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_global_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "ingress_global_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "lf_field_list"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "lf_field_list"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "mcast_grp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "mcast_grp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "resubmit_flag"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "resubmit_flag"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_rid"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_3", "egress_rid"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 8,
-            "column" : 49,
-            "source_fragment" : "standard_metadata, Port port) { ..."
-          }
-        }
-      ]
-    },
-    {
-      "name" : "send_to_cpu",
-      "id" : 1,
-      "runtime_data" : [],
-      "primitives" : [
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "ingress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "egress_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "egress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "clone_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "clone_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "instance_type"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "instance_type"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "drop"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "drop"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "recirculate_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "recirculate_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "packet_length"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "packet_length"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "enq_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "enq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "deq_timedelta"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_timedelta"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "deq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "ingress_global_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_global_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "lf_field_list"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "lf_field_list"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "mcast_grp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "mcast_grp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "resubmit_flag"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "resubmit_flag"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "egress_rid"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_rid"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "egress_spec"]
-            },
-            {
-              "type" : "hexstr",
-              "value" : "0x00ff"
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 5,
-            "column" : 4,
-            "source_fragment" : "standard_metadata.egress_spec = 9w255"
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "ingress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "egress_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "egress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "clone_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "clone_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "instance_type"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "instance_type"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "drop"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "drop"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "recirculate_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "recirculate_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "packet_length"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "packet_length"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "enq_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "enq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_timedelta"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "deq_timedelta"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "deq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_global_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "ingress_global_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "lf_field_list"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "lf_field_list"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "mcast_grp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "mcast_grp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "resubmit_flag"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "resubmit_flag"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_rid"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_4", "egress_rid"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 4,
-            "column" : 45,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        }
-      ]
-    },
-    {
-      "name" : "drop",
-      "id" : 2,
-      "runtime_data" : [],
-      "primitives" : [
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "ingress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "egress_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "egress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "clone_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "clone_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "instance_type"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "instance_type"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "drop"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "drop"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "recirculate_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "recirculate_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "packet_length"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "packet_length"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "enq_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "enq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "deq_timedelta"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_timedelta"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "deq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "ingress_global_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_global_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "lf_field_list"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "lf_field_list"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "mcast_grp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "mcast_grp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "resubmit_flag"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "resubmit_flag"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "egress_rid"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_rid"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "egress_spec"]
-            },
-            {
-              "type" : "hexstr",
-              "value" : "0x01ff"
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 13,
-            "column" : 4,
-            "source_fragment" : "standard_metadata.egress_spec = 9w511"
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "ingress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "egress_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "egress_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "clone_spec"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "clone_spec"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "instance_type"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "instance_type"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "drop"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "drop"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "recirculate_port"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "recirculate_port"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "packet_length"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "packet_length"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "enq_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "enq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "enq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_timedelta"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "deq_timedelta"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "deq_qdepth"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "deq_qdepth"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "ingress_global_timestamp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "ingress_global_timestamp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "lf_field_list"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "lf_field_list"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "mcast_grp"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "mcast_grp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "resubmit_flag"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "resubmit_flag"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["standard_metadata", "egress_rid"]
-            },
-            {
-              "type" : "field",
-              "value" : ["standard_metadata_5", "egress_rid"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/actions.p4",
-            "line" : 12,
-            "column" : 38,
-            "source_fragment" : "standard_metadata) { ..."
-          }
-        }
-      ]
-    },
-    {
-      "name" : "NoAction",
-      "id" : 3,
-      "runtime_data" : [],
-      "primitives" : []
-    },
-    {
-      "name" : "act",
-      "id" : 4,
-      "runtime_data" : [],
-      "primitives" : [
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["scalars", "tmp"]
-            },
-            {
-              "type" : "expression",
-              "value" : {
-                "type" : "expression",
-                "value" : {
-                  "op" : "&",
-                  "left" : {
-                    "type" : "field",
-                    "value" : ["standard_metadata", "ingress_port"]
-                  },
-                  "right" : {
-                    "type" : "hexstr",
-                    "value" : "0xffffffff"
-                  }
-                }
-              }
-            }
-          ]
-        },
-        {
-          "op" : "count",
-          "parameters" : [
-            {
-              "type" : "counter_array",
-              "value" : "port_counters_control.ingress_port_counter"
-            },
-            {
-              "type" : "field",
-              "value" : ["scalars", "tmp"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/port_counters.p4",
-            "line" : 11,
-            "column" : 12,
-            "source_fragment" : "ingress_port_counter.count((bit<32>)standard_metadata.ingress_port)"
-          }
-        },
-        {
-          "op" : "assign",
-          "parameters" : [
-            {
-              "type" : "field",
-              "value" : ["scalars", "tmp_0"]
-            },
-            {
-              "type" : "expression",
-              "value" : {
-                "type" : "expression",
-                "value" : {
-                  "op" : "&",
-                  "left" : {
-                    "type" : "field",
-                    "value" : ["standard_metadata", "egress_spec"]
-                  },
-                  "right" : {
-                    "type" : "hexstr",
-                    "value" : "0xffffffff"
-                  }
-                }
-              }
-            }
-          ]
-        },
-        {
-          "op" : "count",
-          "parameters" : [
-            {
-              "type" : "counter_array",
-              "value" : "port_counters_control.egress_port_counter"
-            },
-            {
-              "type" : "field",
-              "value" : ["scalars", "tmp_0"]
-            }
-          ],
-          "source_info" : {
-            "filename" : "include/port_counters.p4",
-            "line" : 12,
-            "column" : 12,
-            "source_fragment" : "egress_port_counter.count((bit<32>)standard_metadata.egress_spec)"
-          }
-        }
-      ]
-    }
-  ],
-  "pipelines" : [
-    {
-      "name" : "ingress",
-      "id" : 0,
-      "source_info" : {
-        "filename" : "default.p4",
-        "line" : 10,
-        "column" : 8,
-        "source_fragment" : "ingress"
-      },
-      "init_table" : "table0",
-      "tables" : [
-        {
-          "name" : "table0",
-          "id" : 0,
-          "source_info" : {
-            "filename" : "default.p4",
-            "line" : 13,
-            "column" : 10,
-            "source_fragment" : "table0"
-          },
-          "key" : [
-            {
-              "match_type" : "ternary",
-              "target" : ["standard_metadata", "ingress_port"],
-              "mask" : null
-            },
-            {
-              "match_type" : "ternary",
-              "target" : ["ethernet", "dstAddr"],
-              "mask" : null
-            },
-            {
-              "match_type" : "ternary",
-              "target" : ["ethernet", "srcAddr"],
-              "mask" : null
-            },
-            {
-              "match_type" : "ternary",
-              "target" : ["ethernet", "etherType"],
-              "mask" : null
-            }
-          ],
-          "match_type" : "ternary",
-          "type" : "simple",
-          "max_size" : 1024,
-          "support_timeout" : false,
-          "direct_meters" : null,
-          "action_ids" : [0, 1, 2, 3],
-          "actions" : ["set_egress_port", "send_to_cpu", "drop", "NoAction"],
-          "base_default_next" : "node_3",
-          "next_tables" : {
-            "set_egress_port" : "node_3",
-            "send_to_cpu" : "node_3",
-            "drop" : "node_3",
-            "NoAction" : "node_3"
-          },
-          "default_entry" : {
-            "action_id" : 3,
-            "action_const" : false,
-            "action_data" : [],
-            "action_entry_const" : false
-          }
-        },
-        {
-          "name" : "tbl_act",
-          "id" : 1,
-          "key" : [],
-          "match_type" : "exact",
-          "type" : "simple",
-          "max_size" : 1024,
-          "with_counters" : false,
-          "support_timeout" : false,
-          "direct_meters" : null,
-          "action_ids" : [4],
-          "actions" : ["act"],
-          "base_default_next" : null,
-          "next_tables" : {
-            "act" : null
-          },
-          "default_entry" : {
-            "action_id" : 4,
-            "action_const" : true,
-            "action_data" : [],
-            "action_entry_const" : true
-          }
-        }
-      ],
-      "action_profiles" : [],
-      "conditionals" : [
-        {
-          "name" : "node_3",
-          "id" : 0,
-          "source_info" : {
-            "filename" : "include/port_counters.p4",
-            "line" : 10,
-            "column" : 12,
-            "source_fragment" : "standard_metadata.egress_spec < 254"
-          },
-          "expression" : {
-            "type" : "expression",
-            "value" : {
-              "op" : "<",
-              "left" : {
-                "type" : "field",
-                "value" : ["standard_metadata", "egress_spec"]
-              },
-              "right" : {
-                "type" : "hexstr",
-                "value" : "0x00fe"
-              }
-            }
-          },
-          "false_next" : null,
-          "true_next" : "tbl_act"
-        }
-      ]
-    },
-    {
-      "name" : "egress",
-      "id" : 1,
-      "source_info" : {
-        "filename" : "default.p4",
-        "line" : 36,
-        "column" : 8,
-        "source_fragment" : "egress"
-      },
-      "init_table" : null,
-      "tables" : [],
-      "action_profiles" : [],
-      "conditionals" : []
-    }
-  ],
-  "checksums" : [],
-  "force_arith" : [],
-  "extern_instances" : [],
-  "field_aliases" : [
-    [
-      "queueing_metadata.enq_timestamp",
-      ["standard_metadata", "enq_timestamp"]
-    ],
-    [
-      "queueing_metadata.enq_qdepth",
-      ["standard_metadata", "enq_qdepth"]
-    ],
-    [
-      "queueing_metadata.deq_timedelta",
-      ["standard_metadata", "deq_timedelta"]
-    ],
-    [
-      "queueing_metadata.deq_qdepth",
-      ["standard_metadata", "deq_qdepth"]
-    ],
-    [
-      "intrinsic_metadata.ingress_global_timestamp",
-      ["standard_metadata", "ingress_global_timestamp"]
-    ],
-    [
-      "intrinsic_metadata.lf_field_list",
-      ["standard_metadata", "lf_field_list"]
-    ],
-    [
-      "intrinsic_metadata.mcast_grp",
-      ["standard_metadata", "mcast_grp"]
-    ],
-    [
-      "intrinsic_metadata.resubmit_flag",
-      ["standard_metadata", "resubmit_flag"]
-    ],
-    [
-      "intrinsic_metadata.egress_rid",
-      ["standard_metadata", "egress_rid"]
-    ]
-  ]
-}
\ No newline at end of file
diff --git a/protocols/p4runtime/ctl/src/test/resources/default.p4info b/protocols/p4runtime/ctl/src/test/resources/default.p4info
deleted file mode 100644
index b224b87..0000000
--- a/protocols/p4runtime/ctl/src/test/resources/default.p4info
+++ /dev/null
@@ -1,147 +0,0 @@
-tables {
-  preamble {
-    id: 33617813
-    name: "table0"
-    alias: "table0"
-  }
-  match_fields {
-    id: 1
-    name: "standard_metadata.ingress_port"
-    bitwidth: 9
-    match_type: TERNARY
-  }
-  match_fields {
-    id: 2
-    name: "hdr.ethernet.dstAddr"
-    bitwidth: 48
-    match_type: TERNARY
-  }
-  match_fields {
-    id: 3
-    name: "hdr.ethernet.srcAddr"
-    bitwidth: 48
-    match_type: TERNARY
-  }
-  match_fields {
-    id: 4
-    name: "hdr.ethernet.etherType"
-    bitwidth: 16
-    match_type: TERNARY
-  }
-  action_refs {
-    id: 16794308
-  }
-  action_refs {
-    id: 16829080
-  }
-  action_refs {
-    id: 16793508
-  }
-  action_refs {
-    id: 16800567
-    annotations: "@defaultonly()"
-  }
-  direct_resource_ids: 301990488
-  size: 1024
-  with_entry_timeout: true
-}
-actions {
-  preamble {
-    id: 16794308
-    name: "set_egress_port"
-    alias: "set_egress_port"
-  }
-  params {
-    id: 1
-    name: "port"
-    bitwidth: 9
-  }
-}
-actions {
-  preamble {
-    id: 16829080
-    name: "send_to_cpu"
-    alias: "send_to_cpu"
-  }
-}
-actions {
-  preamble {
-    id: 16793508
-    name: "drop"
-    alias: "drop"
-  }
-}
-actions {
-  preamble {
-    id: 16800567
-    name: "NoAction"
-    alias: "NoAction"
-  }
-}
-counters {
-  preamble {
-    id: 302025528
-    name: "port_counters_control.egress_port_counter"
-    alias: "egress_port_counter"
-  }
-  spec {
-    unit: PACKETS
-  }
-  size: 254
-}
-counters {
-  preamble {
-    id: 301999025
-    name: "port_counters_control.ingress_port_counter"
-    alias: "ingress_port_counter"
-  }
-  spec {
-    unit: PACKETS
-  }
-  size: 254
-}
-direct_counters {
-  preamble {
-    id: 301990488
-    name: "table0_counter"
-    alias: "table0_counter"
-  }
-  spec {
-    unit: PACKETS
-  }
-  direct_table_id: 33617813
-}
-controller_packet_metadata {
-  preamble {
-    id: 2868941301
-    name: "packet_in"
-    annotations: "@controller_header(\"packet_in\")"
-  }
-  metadata {
-    id: 1
-    name: "ingress_port"
-    bitwidth: 9
-  }
-  metadata {
-    id: 2
-    name: "other1"
-    bitwidth: 32
-  }
-}
-controller_packet_metadata {
-  preamble {
-    id: 2868916615
-    name: "packet_out"
-    annotations: "@controller_header(\"packet_out\")"
-  }
-  metadata {
-    id: 1
-    name: "egress_port"
-    bitwidth: 9
-  }
-  metadata {
-    id: 2
-    name: "other2"
-    bitwidth: 32
-  }
-}
diff --git a/protocols/p4runtime/ctl/src/test/resources/default.p4info b/protocols/p4runtime/ctl/src/test/resources/default.p4info
new file mode 120000
index 0000000..4dda381
--- /dev/null
+++ b/protocols/p4runtime/ctl/src/test/resources/default.p4info
@@ -0,0 +1 @@
+../../../../../../tools/test/p4src/p4-16/p4c-out/default.p4info
\ No newline at end of file