[SDFAB-189] UpfProgrammable implementation for fabric v1model

Change-Id: I4ea7980830d761a0da8a78943c08229c2da9410d
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/FabricPipeconfManager.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/FabricPipeconfManager.java
index d8af395..7d1ee4e 100644
--- a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/FabricPipeconfManager.java
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/FabricPipeconfManager.java
@@ -19,6 +19,8 @@
 import org.onosproject.net.behaviour.inbandtelemetry.IntProgrammable;
 
 import org.onosproject.net.behaviour.BngProgrammable;
+import org.onosproject.net.behaviour.upf.UpfProgrammable;
+import org.onosproject.pipelines.fabric.impl.behaviour.upf.FabricUpfProgrammable;
 import org.onosproject.net.behaviour.Pipeliner;
 import org.onosproject.net.pi.model.DefaultPiPipeconf;
 import org.onosproject.net.pi.model.PiPipeconf;
@@ -51,6 +53,7 @@
     private static final String INT_PROFILE_SUFFIX = "-int";
     private static final String FULL_PROFILE_SUFFIX = "-full";
     private static final String BNG_PROFILE_SUFFIX = "-bng";
+    private static final String UPF_PROFILE_SUFFIX = "-spgw";
 
     private static Logger log = getLogger(FabricPipeconfLoader.class);
 
@@ -98,6 +101,11 @@
         if (profileName.endsWith(BNG_PROFILE_SUFFIX)) {
             pipeconfBuilder.addBehaviour(BngProgrammable.class, FabricBngProgrammable.class);
         }
+        // Add UpfProgrammable behavior for UPF-enabled pipelines.
+        if (profileName.contains(UPF_PROFILE_SUFFIX) ||
+                profileName.endsWith(FULL_PROFILE_SUFFIX)) {
+            pipeconfBuilder.addBehaviour(UpfProgrammable.class, FabricUpfProgrammable.class);
+        }
         return pipeconfBuilder.build();
     }
 
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/FabricCapabilities.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/FabricCapabilities.java
index 2eb947d..fbb709a 100644
--- a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/FabricCapabilities.java
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/FabricCapabilities.java
@@ -28,6 +28,7 @@
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.onosproject.net.pi.model.PiPipeconf.ExtensionType.CPU_PORT_TXT;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_DOWNLINK_PDRS;
 import static org.slf4j.LoggerFactory.getLogger;
 
 /**
@@ -89,6 +90,17 @@
     }
 
     /**
+     * Returns true if the pipeconf supports UPF capabilities, false otherwise.
+     *
+     * @return boolean
+     */
+    public boolean supportUpf() {
+        return pipeconf.pipelineModel()
+                .table(FABRIC_INGRESS_SPGW_DOWNLINK_PDRS)
+                .isPresent();
+    }
+
+    /**
      * Returns true if the pipeconf supports BNG user plane capabilities, false
      * otherwise.
      *
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/DistributedFabricUpfStore.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/DistributedFabricUpfStore.java
new file mode 100644
index 0000000..1f40f3b
--- /dev/null
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/DistributedFabricUpfStore.java
@@ -0,0 +1,248 @@
+/*
+ * Copyright 2021-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.pipelines.fabric.impl.behaviour.upf;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.ImmutableBiMap;
+import com.google.common.collect.Maps;
+import org.onlab.packet.Ip4Address;
+import org.onlab.util.ImmutableByteSequence;
+import org.onlab.util.KryoNamespace;
+import org.onosproject.net.behaviour.upf.PacketDetectionRule;
+import org.onosproject.store.serializers.KryoNamespaces;
+import org.onosproject.store.service.ConsistentMap;
+import org.onosproject.store.service.DistributedSet;
+import org.onosproject.store.service.MapEvent;
+import org.onosproject.store.service.MapEventListener;
+import org.onosproject.store.service.Serializer;
+import org.onosproject.store.service.StorageService;
+import org.osgi.service.component.annotations.Activate;
+import org.osgi.service.component.annotations.Component;
+import org.osgi.service.component.annotations.Deactivate;
+import org.osgi.service.component.annotations.Reference;
+import org.osgi.service.component.annotations.ReferenceCardinality;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Distributed implementation of FabricUpfStore.
+ */
+// FIXME: this store is generic and not tied to a single device, should we have a store based on deviceId?
+@Component(immediate = true, service = FabricUpfStore.class)
+public final class DistributedFabricUpfStore implements FabricUpfStore {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY)
+    protected StorageService storageService;
+
+    protected static final String FAR_ID_MAP_NAME = "fabric-upf-far-id";
+    protected static final String BUFFER_FAR_ID_SET_NAME = "fabric-upf-buffer-far-id";
+    protected static final String FAR_ID_UE_MAP_NAME = "fabric-upf-far-id-ue";
+    protected static final KryoNamespace.Builder SERIALIZER = KryoNamespace.newBuilder()
+            .register(KryoNamespaces.API)
+            .register(UpfRuleIdentifier.class);
+
+    // TODO: check queue IDs for BMv2, is priority inverted?
+    // Mapping between scheduling priority ranges with BMv2 priority queues
+    private static final BiMap<Integer, Integer> SCHEDULING_PRIORITY_MAP
+            = new ImmutableBiMap.Builder<Integer, Integer>()
+            // Highest scheduling priority for 3GPP is 1 and highest BMv2 queue priority is 7
+            .put(1, 5)
+            .put(6, 4)
+            .put(7, 3)
+            .put(8, 2)
+            .put(9, 1)
+            .build();
+
+    // Distributed local FAR ID to global FAR ID mapping
+    protected ConsistentMap<UpfRuleIdentifier, Integer> farIdMap;
+    private MapEventListener<UpfRuleIdentifier, Integer> farIdMapListener;
+    // Local, reversed copy of farIdMapper for better reverse lookup performance
+    protected Map<Integer, UpfRuleIdentifier> reverseFarIdMap;
+    private int nextGlobalFarId = 1;
+
+    protected DistributedSet<UpfRuleIdentifier> bufferFarIds;
+    protected ConsistentMap<UpfRuleIdentifier, Set<Ip4Address>> farIdToUeAddrs;
+
+    @Activate
+    protected void activate() {
+        // Allow unit test to inject farIdMap here.
+        if (storageService != null) {
+            this.farIdMap = storageService.<UpfRuleIdentifier, Integer>consistentMapBuilder()
+                    .withName(FAR_ID_MAP_NAME)
+                    .withRelaxedReadConsistency()
+                    .withSerializer(Serializer.using(SERIALIZER.build()))
+                    .build();
+            this.bufferFarIds = storageService.<UpfRuleIdentifier>setBuilder()
+                    .withName(BUFFER_FAR_ID_SET_NAME)
+                    .withRelaxedReadConsistency()
+                    .withSerializer(Serializer.using(SERIALIZER.build()))
+                    .build().asDistributedSet();
+            this.farIdToUeAddrs = storageService.<UpfRuleIdentifier, Set<Ip4Address>>consistentMapBuilder()
+                    .withName(FAR_ID_UE_MAP_NAME)
+                    .withRelaxedReadConsistency()
+                    .withSerializer(Serializer.using(SERIALIZER.build()))
+                    .build();
+
+        }
+        farIdMapListener = new FarIdMapListener();
+        farIdMap.addListener(farIdMapListener);
+
+        reverseFarIdMap = Maps.newHashMap();
+        farIdMap.entrySet().forEach(entry -> reverseFarIdMap.put(entry.getValue().value(), entry.getKey()));
+
+        log.info("Started");
+    }
+
+    @Deactivate
+    protected void deactivate() {
+        farIdMap.removeListener(farIdMapListener);
+        farIdMap.destroy();
+        reverseFarIdMap.clear();
+
+        log.info("Stopped");
+    }
+
+    @Override
+    public void reset() {
+        farIdMap.clear();
+        reverseFarIdMap.clear();
+        bufferFarIds.clear();
+        farIdToUeAddrs.clear();
+        nextGlobalFarId = 0;
+    }
+
+    @Override
+    public Map<UpfRuleIdentifier, Integer> getFarIdMap() {
+        return Map.copyOf(farIdMap.asJavaMap());
+    }
+
+    @Override
+    public int globalFarIdOf(UpfRuleIdentifier farIdPair) {
+        int globalFarId = farIdMap.compute(farIdPair,
+                (k, existingId) -> {
+                    return Objects.requireNonNullElseGet(existingId, () -> nextGlobalFarId++);
+                }).value();
+        log.info("{} translated to GlobalFarId={}", farIdPair, globalFarId);
+        return globalFarId;
+    }
+
+    @Override
+    public int globalFarIdOf(ImmutableByteSequence pfcpSessionId, int sessionLocalFarId) {
+        UpfRuleIdentifier farId = new UpfRuleIdentifier(pfcpSessionId, sessionLocalFarId);
+        return globalFarIdOf(farId);
+
+    }
+
+    @Override
+    public String queueIdOf(int schedulingPriority) {
+        return (SCHEDULING_PRIORITY_MAP.get(schedulingPriority)).toString();
+    }
+
+    @Override
+    public String schedulingPriorityOf(int queueId) {
+        return (SCHEDULING_PRIORITY_MAP.inverse().get(queueId)).toString();
+    }
+
+    @Override
+    public UpfRuleIdentifier localFarIdOf(int globalFarId) {
+        return reverseFarIdMap.get(globalFarId);
+    }
+
+    public void learnFarIdToUeAddrs(PacketDetectionRule pdr) {
+        UpfRuleIdentifier ruleId = UpfRuleIdentifier.of(pdr.sessionId(), pdr.farId());
+        farIdToUeAddrs.compute(ruleId, (k, set) -> {
+            if (set == null) {
+                set = new HashSet<>();
+            }
+            set.add(pdr.ueAddress());
+            return set;
+        });
+    }
+
+    @Override
+    public boolean isFarIdBuffering(UpfRuleIdentifier farId) {
+        checkNotNull(farId);
+        return bufferFarIds.contains(farId);
+    }
+
+    @Override
+    public void learBufferingFarId(UpfRuleIdentifier farId) {
+        checkNotNull(farId);
+        bufferFarIds.add(farId);
+    }
+
+    @Override
+    public void forgetBufferingFarId(UpfRuleIdentifier farId) {
+        checkNotNull(farId);
+        bufferFarIds.remove(farId);
+    }
+
+    @Override
+    public void forgetUeAddr(Ip4Address ueAddr) {
+        farIdToUeAddrs.keySet().forEach(
+                farId -> farIdToUeAddrs.computeIfPresent(farId, (farIdz, ueAddrs) -> {
+                    ueAddrs.remove(ueAddr);
+                    return ueAddrs;
+                }));
+    }
+
+    @Override
+    public Set<Ip4Address> ueAddrsOfFarId(UpfRuleIdentifier farId) {
+        return farIdToUeAddrs.getOrDefault(farId, Set.of()).value();
+    }
+
+    @Override
+    public Set<UpfRuleIdentifier> getBufferFarIds() {
+        return Set.copyOf(bufferFarIds);
+    }
+
+    @Override
+    public Map<UpfRuleIdentifier, Set<Ip4Address>> getFarIdToUeAddrs() {
+        return Map.copyOf(farIdToUeAddrs.asJavaMap());
+    }
+
+    // NOTE: FarIdMapListener is run on the same thread intentionally in order to ensure that
+    //       reverseFarIdMap update always finishes right after farIdMap is updated
+    private class FarIdMapListener implements MapEventListener<UpfRuleIdentifier, Integer> {
+        @Override
+        public void event(MapEvent<UpfRuleIdentifier, Integer> event) {
+            switch (event.type()) {
+                case INSERT:
+                    reverseFarIdMap.put(event.newValue().value(), event.key());
+                    break;
+                case UPDATE:
+                    reverseFarIdMap.remove(event.oldValue().value());
+                    reverseFarIdMap.put(event.newValue().value(), event.key());
+                    break;
+                case REMOVE:
+                    reverseFarIdMap.remove(event.oldValue().value());
+                    break;
+                default:
+                    break;
+            }
+        }
+    }
+}
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfProgrammable.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfProgrammable.java
new file mode 100644
index 0000000..77ad732
--- /dev/null
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfProgrammable.java
@@ -0,0 +1,668 @@
+/*
+ * Copyright 2021-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.pipelines.fabric.impl.behaviour.upf;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import org.onlab.packet.Ip4Address;
+import org.onlab.packet.Ip4Prefix;
+import org.onosproject.core.ApplicationId;
+import org.onosproject.core.CoreService;
+import org.onosproject.drivers.p4runtime.AbstractP4RuntimeHandlerBehaviour;
+import org.onosproject.net.PortNumber;
+import org.onosproject.net.behaviour.upf.ForwardingActionRule;
+import org.onosproject.net.behaviour.upf.GtpTunnel;
+import org.onosproject.net.behaviour.upf.PacketDetectionRule;
+import org.onosproject.net.behaviour.upf.PdrStats;
+import org.onosproject.net.behaviour.upf.UpfInterface;
+import org.onosproject.net.behaviour.upf.UpfProgrammable;
+import org.onosproject.net.behaviour.upf.UpfProgrammableException;
+import org.onosproject.net.flow.DefaultFlowRule;
+import org.onosproject.net.flow.DefaultTrafficSelector;
+import org.onosproject.net.flow.DefaultTrafficTreatment;
+import org.onosproject.net.flow.FlowEntry;
+import org.onosproject.net.flow.FlowRule;
+import org.onosproject.net.flow.FlowRuleService;
+import org.onosproject.net.flow.criteria.PiCriterion;
+import org.onosproject.net.packet.DefaultOutboundPacket;
+import org.onosproject.net.packet.OutboundPacket;
+import org.onosproject.net.packet.PacketService;
+import org.onosproject.net.pi.model.PiCounterId;
+import org.onosproject.net.pi.model.PiCounterModel;
+import org.onosproject.net.pi.model.PiTableId;
+import org.onosproject.net.pi.model.PiTableModel;
+import org.onosproject.net.pi.runtime.PiCounterCell;
+import org.onosproject.net.pi.runtime.PiCounterCellHandle;
+import org.onosproject.net.pi.runtime.PiCounterCellId;
+import org.onosproject.pipelines.fabric.impl.FabricPipeconfLoader;
+import org.onosproject.pipelines.fabric.impl.behaviour.FabricCapabilities;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.onosproject.net.behaviour.upf.UpfProgrammableException.Type.UNSUPPORTED_OPERATION;
+import static org.onosproject.net.pi.model.PiCounterType.INDIRECT;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_EGRESS_SPGW_PDR_COUNTER;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_DOWNLINK_PDRS;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_FARS;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_INTERFACES;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_PDR_COUNTER;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_UPLINK_PDRS;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_FAR_ID;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_GTPU_IS_VALID;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_IPV4_DST_ADDR;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_TEID;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_TUNNEL_IPV4_DST;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_UE_ADDR;
+
+
+/**
+ * Implementation of a UPF programmable device behavior.
+ */
+public class FabricUpfProgrammable extends AbstractP4RuntimeHandlerBehaviour
+        implements UpfProgrammable {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private static final int DEFAULT_PRIORITY = 128;
+    private static final long DEFAULT_P4_DEVICE_ID = 1;
+
+    protected FlowRuleService flowRuleService;
+    protected PacketService packetService;
+    protected FabricUpfStore fabricUpfStore;
+    protected FabricUpfTranslator upfTranslator;
+
+    private long farTableSize;
+    private long encappedPdrTableSize;
+    private long unencappedPdrTableSize;
+    private long pdrCounterSize;
+
+    private ApplicationId appId;
+
+    // FIXME: remove, buffer drain should be triggered by Up4Service
+    private BufferDrainer bufferDrainer;
+
+    // FIXME: dbuf tunnel should be managed by Up4Service
+    //  Up4Service should be responsible of setting up such tunnel, then transforming FARs for this
+    //  device accordingly. When the tunnel endpoint change, it should be up to Up4Service to update
+    //  the FAR on the device.
+    private GtpTunnel dbufTunnel;
+
+    @Override
+    protected boolean setupBehaviour(String opName) {
+        if (!super.setupBehaviour(opName)) {
+            return false;
+        }
+        flowRuleService = handler().get(FlowRuleService.class);
+        packetService = handler().get(PacketService.class);
+        fabricUpfStore = handler().get(FabricUpfStore.class);
+        upfTranslator = new FabricUpfTranslator(fabricUpfStore);
+        final CoreService coreService = handler().get(CoreService.class);
+        appId = coreService.getAppId(FabricPipeconfLoader.PIPELINE_APP_NAME);
+        if (appId == null) {
+            log.warn("Application ID is null. Cannot initialize behaviour.");
+            return false;
+        }
+
+        var capabilities = new FabricCapabilities(pipeconf);
+        if (!capabilities.supportUpf()) {
+            log.warn("Pipeconf {} on {} does not support UPF capabilities, " +
+                             "cannot perform {}",
+                     pipeconf.id(), deviceId, opName);
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public boolean init() {
+        if (setupBehaviour("init()")) {
+            if (!computeHardwareResourceSizes()) {
+                // error message will be printed by computeHardwareResourceSizes()
+                return false;
+            }
+            log.info("UpfProgrammable initialized for appId {} and deviceId {}", appId, deviceId);
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Grab the capacities for the PDR and FAR tables from the pipeconf. Runs only once, on initialization.
+     *
+     * @return true if resource is fetched successfully, false otherwise.
+     * @throws IllegalStateException when FAR or PDR table can't be found in the pipeline model.
+     */
+    private boolean computeHardwareResourceSizes() {
+        long farTableSize = 0;
+        long encappedPdrTableSize = 0;
+        long unencappedPdrTableSize = 0;
+
+        // Get table sizes of interest
+        for (PiTableModel piTable : pipeconf.pipelineModel().tables()) {
+            if (piTable.id().equals(FABRIC_INGRESS_SPGW_UPLINK_PDRS)) {
+                encappedPdrTableSize = piTable.maxSize();
+            } else if (piTable.id().equals(FABRIC_INGRESS_SPGW_DOWNLINK_PDRS)) {
+                unencappedPdrTableSize = piTable.maxSize();
+            } else if (piTable.id().equals(FABRIC_INGRESS_SPGW_FARS)) {
+                farTableSize = piTable.maxSize();
+            }
+        }
+        if (encappedPdrTableSize == 0) {
+            throw new IllegalStateException("Unable to find uplink PDR table in pipeline model.");
+        }
+        if (unencappedPdrTableSize == 0) {
+            throw new IllegalStateException("Unable to find downlink PDR table in pipeline model.");
+        }
+        if (encappedPdrTableSize != unencappedPdrTableSize) {
+            log.warn("The uplink and downlink PDR tables don't have equal sizes! Using the minimum of the two.");
+        }
+        if (farTableSize == 0) {
+            throw new IllegalStateException("Unable to find FAR table in pipeline model.");
+        }
+        // Get counter sizes of interest
+        long ingressCounterSize = 0;
+        long egressCounterSize = 0;
+        for (PiCounterModel piCounter : pipeconf.pipelineModel().counters()) {
+            if (piCounter.id().equals(FABRIC_INGRESS_SPGW_PDR_COUNTER)) {
+                ingressCounterSize = piCounter.size();
+            } else if (piCounter.id().equals(FABRIC_EGRESS_SPGW_PDR_COUNTER)) {
+                egressCounterSize = piCounter.size();
+            }
+        }
+        if (ingressCounterSize != egressCounterSize) {
+            log.warn("PDR ingress and egress counter sizes are not equal! Using the minimum of the two.");
+        }
+        this.farTableSize = farTableSize;
+        this.encappedPdrTableSize = encappedPdrTableSize;
+        this.unencappedPdrTableSize = unencappedPdrTableSize;
+        this.pdrCounterSize = Math.min(ingressCounterSize, egressCounterSize);
+        return true;
+    }
+
+    @Override
+    public void setBufferDrainer(BufferDrainer drainer) {
+        if (!setupBehaviour("setBufferDrainer()")) {
+            return;
+        }
+        this.bufferDrainer = drainer;
+    }
+
+    @Override
+    public void unsetBufferDrainer() {
+        if (!setupBehaviour("unsetBufferDrainer()")) {
+            return;
+        }
+        this.bufferDrainer = null;
+    }
+
+    @Override
+    public void enablePscEncap(int defaultQfi) throws UpfProgrammableException {
+        throw new UpfProgrammableException("PSC encap is not supported in fabric-v1model",
+                                           UNSUPPORTED_OPERATION);
+    }
+
+    @Override
+    public void disablePscEncap() throws UpfProgrammableException {
+        throw new UpfProgrammableException("PSC encap is not supported in fabric-v1model",
+                                           UNSUPPORTED_OPERATION);
+    }
+
+    @Override
+    public void sendPacketOut(ByteBuffer data) {
+        if (!setupBehaviour("sendPacketOut()")) {
+            return;
+        }
+        final OutboundPacket pkt = new DefaultOutboundPacket(
+                deviceId,
+                // Use TABLE logical port to have pkt routed via pipeline tables.
+                DefaultTrafficTreatment.builder()
+                        .setOutput(PortNumber.TABLE)
+                        .build(),
+                data);
+        packetService.emit(pkt);
+    }
+
+    @Override
+    public void setDbufTunnel(Ip4Address switchAddr, Ip4Address dbufAddr) {
+        if (!setupBehaviour("setDbufTunnel()")) {
+            return;
+        }
+        this.dbufTunnel = GtpTunnel.builder()
+                .setSrc(switchAddr)
+                .setDst(dbufAddr)
+                .setSrcPort((short) 2152)
+                .setTeid(0)
+                .build();
+    }
+
+    @Override
+    public void unsetDbufTunnel() {
+        if (!setupBehaviour("unsetDbufTunnel()")) {
+            return;
+        }
+        this.dbufTunnel = null;
+    }
+
+    /**
+     * Convert the given buffering FAR to a FAR that tunnels the packet to dbuf.
+     *
+     * @param far the FAR to convert
+     * @return the converted FAR
+     */
+    private ForwardingActionRule convertToDbufFar(ForwardingActionRule far) {
+        if (!far.buffers()) {
+            throw new IllegalArgumentException("Converting a non-buffering FAR to a dbuf FAR! This shouldn't happen.");
+        }
+        return ForwardingActionRule.builder()
+                .setFarId(far.farId())
+                .withSessionId(far.sessionId())
+                .setNotifyFlag(far.notifies())
+                .setBufferFlag(true)
+                .setTunnel(dbufTunnel)
+                .build();
+    }
+
+    @Override
+    public void cleanUp() {
+        if (!setupBehaviour("cleanUp()")) {
+            return;
+        }
+        log.info("Clearing all UPF-related table entries.");
+        flowRuleService.removeFlowRulesById(appId);
+        fabricUpfStore.reset();
+    }
+
+    @Override
+    public void clearInterfaces() {
+        if (!setupBehaviour("clearInterfaces()")) {
+            return;
+        }
+        log.info("Clearing all UPF interfaces.");
+        for (FlowRule entry : flowRuleService.getFlowEntriesById(appId)) {
+            if (upfTranslator.isFabricInterface(entry)) {
+                flowRuleService.removeFlowRules(entry);
+            }
+        }
+    }
+
+    @Override
+    public void clearFlows() {
+        if (!setupBehaviour("clearFlows()")) {
+            return;
+        }
+        log.info("Clearing all UE sessions.");
+        int pdrsCleared = 0;
+        int farsCleared = 0;
+        for (FlowRule entry : flowRuleService.getFlowEntriesById(appId)) {
+            if (upfTranslator.isFabricPdr(entry)) {
+                pdrsCleared++;
+                flowRuleService.removeFlowRules(entry);
+            } else if (upfTranslator.isFabricFar(entry)) {
+                farsCleared++;
+                flowRuleService.removeFlowRules(entry);
+            }
+        }
+        log.info("Cleared {} PDRs and {} FARS.", pdrsCleared, farsCleared);
+    }
+
+
+    @Override
+    public Collection<PdrStats> readAllCounters(long maxCounterId) {
+        if (!setupBehaviour("readAllCounters()")) {
+            return null;
+        }
+
+        long counterSize = pdrCounterSize();
+        if (maxCounterId != -1) {
+            counterSize = Math.min(maxCounterId, counterSize);
+        }
+
+        // Prepare PdrStats object builders, one for each counter ID currently in use
+        Map<Integer, PdrStats.Builder> pdrStatBuilders = Maps.newHashMap();
+        for (int cellId = 0; cellId < counterSize; cellId++) {
+            pdrStatBuilders.put(cellId, PdrStats.builder().withCellId(cellId));
+        }
+
+        // Generate the counter cell IDs.
+        Set<PiCounterId> counterIds = ImmutableSet.of(
+                FABRIC_INGRESS_SPGW_PDR_COUNTER,
+                FABRIC_EGRESS_SPGW_PDR_COUNTER
+        );
+
+        // Query the device.
+        Collection<PiCounterCell> counterEntryResponse = client.read(
+                DEFAULT_P4_DEVICE_ID, pipeconf)
+                .counterCells(counterIds)
+                .submitSync()
+                .all(PiCounterCell.class);
+
+        // Process response.
+        counterEntryResponse.forEach(counterCell -> {
+            if (counterCell.cellId().counterType() != INDIRECT) {
+                log.warn("Invalid counter data type {}, skipping", counterCell.cellId().counterType());
+                return;
+            }
+            if (!pdrStatBuilders.containsKey((int) counterCell.cellId().index())) {
+                // Most likely Up4config.maxUes() is set to a value smaller than what the switch
+                // pipeline can hold.
+                log.debug("Unrecognized index {} when reading all counters, " +
+                                  "that's expected if we are manually limiting maxUes", counterCell);
+                return;
+            }
+            PdrStats.Builder statsBuilder = pdrStatBuilders.get((int) counterCell.cellId().index());
+            if (counterCell.cellId().counterId().equals(FABRIC_INGRESS_SPGW_PDR_COUNTER)) {
+                statsBuilder.setIngress(counterCell.data().packets(),
+                                        counterCell.data().bytes());
+            } else if (counterCell.cellId().counterId().equals(FABRIC_EGRESS_SPGW_PDR_COUNTER)) {
+                statsBuilder.setEgress(counterCell.data().packets(),
+                                       counterCell.data().bytes());
+            } else {
+                log.warn("Unrecognized counter ID {}, skipping", counterCell);
+            }
+        });
+
+        return pdrStatBuilders
+                .values()
+                .stream()
+                .map(PdrStats.Builder::build)
+                .collect(Collectors.toList());
+    }
+
+    @Override
+    public long pdrCounterSize() {
+        if (!setupBehaviour("pdrCounterSize()")) {
+            return -1;
+        }
+        computeHardwareResourceSizes();
+        return pdrCounterSize;
+    }
+
+    @Override
+    public long farTableSize() {
+        if (!setupBehaviour("farTableSize()")) {
+            return -1;
+        }
+        computeHardwareResourceSizes();
+        return farTableSize;
+    }
+
+    @Override
+    public long pdrTableSize() {
+        if (!setupBehaviour("pdrTableSize()")) {
+            return -1;
+        }
+        computeHardwareResourceSizes();
+        return Math.min(encappedPdrTableSize, unencappedPdrTableSize) * 2;
+    }
+
+    @Override
+    public PdrStats readCounter(int cellId) throws UpfProgrammableException {
+        if (!setupBehaviour("readCounter()")) {
+            return null;
+        }
+        if (cellId >= pdrCounterSize() || cellId < 0) {
+            throw new UpfProgrammableException("Requested PDR counter cell index is out of bounds.",
+                                               UpfProgrammableException.Type.COUNTER_INDEX_OUT_OF_RANGE);
+        }
+        PdrStats.Builder stats = PdrStats.builder().withCellId(cellId);
+
+        // Make list of cell handles we want to read.
+        List<PiCounterCellHandle> counterCellHandles = List.of(
+                PiCounterCellHandle.of(deviceId,
+                                       PiCounterCellId.ofIndirect(FABRIC_INGRESS_SPGW_PDR_COUNTER, cellId)),
+                PiCounterCellHandle.of(deviceId,
+                                       PiCounterCellId.ofIndirect(FABRIC_EGRESS_SPGW_PDR_COUNTER, cellId)));
+
+        // Query the device.
+        Collection<PiCounterCell> counterEntryResponse = client.read(
+                DEFAULT_P4_DEVICE_ID, pipeconf)
+                .handles(counterCellHandles).submitSync()
+                .all(PiCounterCell.class);
+
+        // Process response.
+        counterEntryResponse.forEach(counterCell -> {
+            if (counterCell.cellId().counterType() != INDIRECT) {
+                log.warn("Invalid counter data type {}, skipping", counterCell.cellId().counterType());
+                return;
+            }
+            if (cellId != counterCell.cellId().index()) {
+                log.warn("Unrecognized counter index {}, skipping", counterCell);
+                return;
+            }
+            if (counterCell.cellId().counterId().equals(FABRIC_INGRESS_SPGW_PDR_COUNTER)) {
+                stats.setIngress(counterCell.data().packets(), counterCell.data().bytes());
+            } else if (counterCell.cellId().counterId().equals(FABRIC_EGRESS_SPGW_PDR_COUNTER)) {
+                stats.setEgress(counterCell.data().packets(), counterCell.data().bytes());
+            } else {
+                log.warn("Unrecognized counter ID {}, skipping", counterCell);
+            }
+        });
+        return stats.build();
+    }
+
+
+    @Override
+    public void addPdr(PacketDetectionRule pdr) throws UpfProgrammableException {
+        if (!setupBehaviour("addPdr()")) {
+            return;
+        }
+        if (pdr.counterId() >= pdrCounterSize() || pdr.counterId() < 0) {
+            throw new UpfProgrammableException("Counter cell index referenced by PDR is out of bounds.",
+                                               UpfProgrammableException.Type.COUNTER_INDEX_OUT_OF_RANGE);
+        }
+        FlowRule fabricPdr = upfTranslator.pdrToFabricEntry(pdr, deviceId, appId, DEFAULT_PRIORITY);
+        log.info("Installing {}", pdr.toString());
+        flowRuleService.applyFlowRules(fabricPdr);
+        log.debug("PDR added with flowID {}", fabricPdr.id().value());
+
+        // If the flow rule was applied and the PDR is downlink, add the PDR to the farID->PDR mapping
+        if (pdr.matchesUnencapped()) {
+            fabricUpfStore.learnFarIdToUeAddrs(pdr);
+        }
+    }
+
+
+    @Override
+    public void addFar(ForwardingActionRule far) throws UpfProgrammableException {
+        if (!setupBehaviour("addFar()")) {
+            return;
+        }
+        UpfRuleIdentifier ruleId = UpfRuleIdentifier.of(far.sessionId(), far.farId());
+        if (far.buffers()) {
+            // If the far has the buffer flag, modify its tunnel so it directs to dbuf
+            far = convertToDbufFar(far);
+            fabricUpfStore.learBufferingFarId(ruleId);
+        }
+        FlowRule fabricFar = upfTranslator.farToFabricEntry(far, deviceId, appId, DEFAULT_PRIORITY);
+        log.info("Installing {}", far.toString());
+        flowRuleService.applyFlowRules(fabricFar);
+        log.debug("FAR added with flowID {}", fabricFar.id().value());
+        if (!far.buffers() && fabricUpfStore.isFarIdBuffering(ruleId)) {
+            // If this FAR does not buffer but used to, then drain the buffer for every UE address
+            // that hits this FAR.
+            fabricUpfStore.forgetBufferingFarId(ruleId);
+            for (var ueAddr : fabricUpfStore.ueAddrsOfFarId(ruleId)) {
+                if (bufferDrainer == null) {
+                    log.warn("Unable to drain downlink buffer for UE {}, bufferDrainer is null", ueAddr);
+                } else {
+                    bufferDrainer.drain(ueAddr);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void addInterface(UpfInterface upfInterface) throws UpfProgrammableException {
+        if (!setupBehaviour("addInterface()")) {
+            return;
+        }
+        FlowRule flowRule = upfTranslator.interfaceToFabricEntry(upfInterface, deviceId, appId, DEFAULT_PRIORITY);
+        log.info("Installing {}", upfInterface);
+        flowRuleService.applyFlowRules(flowRule);
+        log.debug("Interface added with flowID {}", flowRule.id().value());
+        // By default we enable UE-to-UE communication on the UE subnet identified by the CORE interface.
+        // TODO: allow enabling/disabling UE-to-UE via netcfg or other API.
+        log.warn("UE-to-UE traffic is not supported in fabric-v1model");
+    }
+
+    private boolean removeEntry(PiCriterion match, PiTableId tableId, boolean failSilent)
+            throws UpfProgrammableException {
+        if (!setupBehaviour("removeEntry()")) {
+            return false;
+        }
+        FlowRule entry = DefaultFlowRule.builder()
+                .forDevice(deviceId).fromApp(appId).makePermanent()
+                .forTable(tableId)
+                .withSelector(DefaultTrafficSelector.builder().matchPi(match).build())
+                .withPriority(DEFAULT_PRIORITY)
+                .build();
+
+        /*
+         *  FIXME: Stupid stupid slow hack, needed because removeFlowRules expects FlowRule objects
+         *   with correct and complete actions and parameters, but P4Runtime deletion requests
+         *   will not have those.
+         */
+        for (FlowEntry installedEntry : flowRuleService.getFlowEntriesById(appId)) {
+            if (installedEntry.selector().equals(entry.selector())) {
+                log.info("Found matching entry to remove, it has FlowID {}", installedEntry.id());
+                flowRuleService.removeFlowRules(installedEntry);
+                return true;
+            }
+        }
+        if (!failSilent) {
+            throw new UpfProgrammableException("Match criterion " + match.toString() +
+                                                       " not found in table " + tableId.toString());
+        }
+        return false;
+    }
+
+    @Override
+    public Collection<PacketDetectionRule> getPdrs() throws UpfProgrammableException {
+        if (!setupBehaviour("getPdrs()")) {
+            return null;
+        }
+        ArrayList<PacketDetectionRule> pdrs = new ArrayList<>();
+        for (FlowRule flowRule : flowRuleService.getFlowEntriesById(appId)) {
+            if (upfTranslator.isFabricPdr(flowRule)) {
+                pdrs.add(upfTranslator.fabricEntryToPdr(flowRule));
+            }
+        }
+        return pdrs;
+    }
+
+    @Override
+    public Collection<ForwardingActionRule> getFars() throws UpfProgrammableException {
+        if (!setupBehaviour("getFars()")) {
+            return null;
+        }
+        ArrayList<ForwardingActionRule> fars = new ArrayList<>();
+        for (FlowRule flowRule : flowRuleService.getFlowEntriesById(appId)) {
+            if (upfTranslator.isFabricFar(flowRule)) {
+                fars.add(upfTranslator.fabricEntryToFar(flowRule));
+            }
+        }
+        return fars;
+    }
+
+    @Override
+    public Collection<UpfInterface> getInterfaces() throws UpfProgrammableException {
+        if (!setupBehaviour("getInterfaces()")) {
+            return null;
+        }
+        ArrayList<UpfInterface> ifaces = new ArrayList<>();
+        for (FlowRule flowRule : flowRuleService.getFlowEntriesById(appId)) {
+            if (upfTranslator.isFabricInterface(flowRule)) {
+                ifaces.add(upfTranslator.fabricEntryToInterface(flowRule));
+            }
+        }
+        return ifaces;
+    }
+
+    @Override
+    public void removePdr(PacketDetectionRule pdr) throws UpfProgrammableException {
+        if (!setupBehaviour("removePdr()")) {
+            return;
+        }
+        final PiCriterion match;
+        final PiTableId tableId;
+        if (pdr.matchesEncapped()) {
+            match = PiCriterion.builder()
+                    .matchExact(HDR_TEID, pdr.teid().asArray())
+                    .matchExact(HDR_TUNNEL_IPV4_DST, pdr.tunnelDest().toInt())
+                    .build();
+            tableId = FABRIC_INGRESS_SPGW_UPLINK_PDRS;
+        } else {
+            match = PiCriterion.builder()
+                    .matchExact(HDR_UE_ADDR, pdr.ueAddress().toInt())
+                    .build();
+            tableId = FABRIC_INGRESS_SPGW_DOWNLINK_PDRS;
+        }
+        log.info("Removing {}", pdr.toString());
+        removeEntry(match, tableId, false);
+
+        // Remove the PDR from the farID->PDR mapping
+        // This is an inefficient hotfix FIXME: remove UE addrs from the mapping in sublinear time
+        if (pdr.matchesUnencapped()) {
+            // Should we remove just from the map entry with key == far ID?
+            fabricUpfStore.forgetUeAddr(pdr.ueAddress());
+        }
+    }
+
+    @Override
+    public void removeFar(ForwardingActionRule far) throws UpfProgrammableException {
+        log.info("Removing {}", far.toString());
+
+        PiCriterion match = PiCriterion.builder()
+                .matchExact(HDR_FAR_ID, fabricUpfStore.globalFarIdOf(far.sessionId(), far.farId()))
+                .build();
+
+        removeEntry(match, FABRIC_INGRESS_SPGW_FARS, false);
+    }
+
+    @Override
+    public void removeInterface(UpfInterface upfInterface) throws UpfProgrammableException {
+        if (!setupBehaviour("removeInterface()")) {
+            return;
+        }
+        Ip4Prefix ifacePrefix = upfInterface.getPrefix();
+        // If it isn't a core interface (so it is either access or unknown), try removing core
+        if (!upfInterface.isCore()) {
+            PiCriterion match1 = PiCriterion.builder()
+                    .matchLpm(HDR_IPV4_DST_ADDR, ifacePrefix.address().toInt(),
+                              ifacePrefix.prefixLength())
+                    .matchExact(HDR_GTPU_IS_VALID, 1)
+                    .build();
+            if (removeEntry(match1, FABRIC_INGRESS_SPGW_INTERFACES, true)) {
+                return;
+            }
+        }
+        // If that didn't work or didn't execute, try removing access
+        PiCriterion match2 = PiCriterion.builder()
+                .matchLpm(HDR_IPV4_DST_ADDR, ifacePrefix.address().toInt(),
+                          ifacePrefix.prefixLength())
+                .matchExact(HDR_GTPU_IS_VALID, 0)
+                .build();
+        removeEntry(match2, FABRIC_INGRESS_SPGW_INTERFACES, false);
+    }
+}
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfStore.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfStore.java
new file mode 100644
index 0000000..786ef3c
--- /dev/null
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfStore.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2021-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.pipelines.fabric.impl.behaviour.upf;
+
+import org.onlab.packet.Ip4Address;
+import org.onlab.util.ImmutableByteSequence;
+import org.onosproject.net.behaviour.upf.PacketDetectionRule;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Stores state required for translation of UPF entities to pipeline-specific ones.
+ */
+public interface FabricUpfStore {
+    /**
+     * Clear all state associated with translation.
+     */
+    void reset();
+
+    /**
+     * Returns the farIdMap.
+     *
+     * @return the farIdMap.
+     */
+    Map<UpfRuleIdentifier, Integer> getFarIdMap();
+
+    /**
+     * Get a globally unique integer identifier for the FAR identified by the given (Session ID, Far
+     * ID) pair.
+     *
+     * @param farIdPair a RuleIdentifier instance uniquely identifying the FAR
+     * @return A globally unique integer identifier
+     */
+    int globalFarIdOf(UpfRuleIdentifier farIdPair);
+
+    /**
+     * Get a globally unique integer identifier for the FAR identified by the given (Session ID, Far
+     * ID) pair.
+     *
+     * @param pfcpSessionId     The ID of the PFCP session that produced the FAR ID.
+     * @param sessionLocalFarId The FAR ID.
+     * @return A globally unique integer identifier
+     */
+    int globalFarIdOf(ImmutableByteSequence pfcpSessionId, int sessionLocalFarId);
+
+    /**
+     * Get the corresponding PFCP session ID and session-local FAR ID from a globally unique FAR ID,
+     * or return null if no such mapping is found.
+     *
+     * @param globalFarId globally unique FAR ID
+     * @return the corresponding PFCP session ID and session-local FAR ID, as a RuleIdentifier
+     */
+    UpfRuleIdentifier localFarIdOf(int globalFarId);
+
+    /**
+     * Get the corresponding queue Id from scheduling priority.
+     *
+     * @param schedulingPriority QCI scheduling priority
+     * @return the corresponding queue ID
+     */
+    String queueIdOf(int schedulingPriority);
+
+    /**
+     * Get the corresponding queue Id from scheduling priority.
+     *
+     * @param queueId Tofino queue Id
+     * @return the corresponding scheduling priroity
+    */
+    String schedulingPriorityOf(int queueId);
+
+    /**
+     * Stores the mapping between FAR ID and UE address as defined by the given PDR.
+     *
+     * @param pdr PDR
+     */
+    void learnFarIdToUeAddrs(PacketDetectionRule pdr);
+
+    /**
+     * Returns true if the given FAR IDs is known to be a buffering one.
+     *
+     * @param farId FAR ID
+     * @return boolean
+     */
+    boolean isFarIdBuffering(UpfRuleIdentifier farId);
+
+    /**
+     * Learns the given FAR ID as being a buffering one.
+     *
+     * @param farId FAR ID
+     */
+    void learBufferingFarId(UpfRuleIdentifier farId);
+
+    /**
+     * Forgets the given FAR ID as being a buffering one.
+     *
+     * @param farId FAR ID
+     */
+    void forgetBufferingFarId(UpfRuleIdentifier farId);
+
+    /**
+     * Returns the set of UE addresses associated with the given FAR ID.
+     *
+     * @param farId FAR ID
+     * @return Set of Ip4Address
+     */
+    Set<Ip4Address> ueAddrsOfFarId(UpfRuleIdentifier farId);
+
+    /**
+     * Removes the given UE address from the FAR ID to UE address map.
+     * @param ueAddr UE address
+     */
+    void forgetUeAddr(Ip4Address ueAddr);
+
+    /**
+     * Returns the set of known buffering FAR IDs.
+     * @return set
+     */
+    Set<UpfRuleIdentifier> getBufferFarIds();
+
+    /**
+     * Returns the FAR ID to UE addresses map.
+     *
+     * @return map
+     */
+    Map<UpfRuleIdentifier, Set<Ip4Address>> getFarIdToUeAddrs();
+}
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfTranslator.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfTranslator.java
new file mode 100644
index 0000000..d270371
--- /dev/null
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfTranslator.java
@@ -0,0 +1,441 @@
+/*
+ * Copyright 2021-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.pipelines.fabric.impl.behaviour.upf;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.onlab.packet.Ip4Address;
+import org.onlab.util.ImmutableByteSequence;
+import org.onosproject.core.ApplicationId;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.behaviour.upf.ForwardingActionRule;
+import org.onosproject.net.behaviour.upf.GtpTunnel;
+import org.onosproject.net.behaviour.upf.PacketDetectionRule;
+import org.onosproject.net.behaviour.upf.UpfInterface;
+import org.onosproject.net.behaviour.upf.UpfProgrammableException;
+import org.onosproject.net.flow.DefaultFlowRule;
+import org.onosproject.net.flow.DefaultTrafficSelector;
+import org.onosproject.net.flow.DefaultTrafficTreatment;
+import org.onosproject.net.flow.FlowRule;
+import org.onosproject.net.flow.criteria.PiCriterion;
+import org.onosproject.net.pi.model.PiActionId;
+import org.onosproject.net.pi.model.PiTableId;
+import org.onosproject.net.pi.runtime.PiAction;
+import org.onosproject.net.pi.runtime.PiActionParam;
+import org.onosproject.net.pi.runtime.PiTableAction;
+
+import java.util.Arrays;
+
+import static org.onosproject.pipelines.fabric.FabricConstants.CTR_ID;
+import static org.onosproject.pipelines.fabric.FabricConstants.DROP;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_DOWNLINK_PDRS;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_FARS;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_INTERFACES;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_LOAD_DBUF_FAR;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_LOAD_IFACE;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_LOAD_NORMAL_FAR;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_LOAD_PDR;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_LOAD_PDR_QOS;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_LOAD_TUNNEL_FAR;
+import static org.onosproject.pipelines.fabric.FabricConstants.FABRIC_INGRESS_SPGW_UPLINK_PDRS;
+import static org.onosproject.pipelines.fabric.FabricConstants.FAR_ID;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_FAR_ID;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_GTPU_IS_VALID;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_IPV4_DST_ADDR;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_TEID;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_TUNNEL_IPV4_DST;
+import static org.onosproject.pipelines.fabric.FabricConstants.HDR_UE_ADDR;
+import static org.onosproject.pipelines.fabric.FabricConstants.NEEDS_GTPU_DECAP;
+import static org.onosproject.pipelines.fabric.FabricConstants.NOTIFY_CP;
+import static org.onosproject.pipelines.fabric.FabricConstants.QID;
+import static org.onosproject.pipelines.fabric.FabricConstants.SRC_IFACE;
+import static org.onosproject.pipelines.fabric.FabricConstants.TEID;
+import static org.onosproject.pipelines.fabric.FabricConstants.TUNNEL_DST_ADDR;
+import static org.onosproject.pipelines.fabric.FabricConstants.TUNNEL_SRC_ADDR;
+import static org.onosproject.pipelines.fabric.FabricConstants.TUNNEL_SRC_PORT;
+
+/**
+ * Provides logic to translate UPF entities into pipeline-specific ones and vice-versa.
+ * Implementation should be stateless, with all state delegated to FabricUpfStore.
+ */
+public class FabricUpfTranslator {
+
+    // UPF related constants
+    public static final int INTERFACE_ACCESS = 1;
+    public static final int INTERFACE_CORE = 2;
+    public static final int INTERFACE_DBUF = 3;
+
+    private final FabricUpfStore fabricUpfStore;
+
+    public FabricUpfTranslator(FabricUpfStore fabricUpfStore) {
+        this.fabricUpfStore = fabricUpfStore;
+    }
+
+    /**
+     * Returns true if the given table entry is a Packet Detection Rule from the physical fabric pipeline, and
+     * false otherwise.
+     *
+     * @param entry the entry that may or may not be a fabric.p4 PDR
+     * @return true if the entry is a fabric.p4 PDR
+     */
+    public boolean isFabricPdr(FlowRule entry) {
+        return entry.table().equals(FABRIC_INGRESS_SPGW_UPLINK_PDRS)
+                || entry.table().equals(FABRIC_INGRESS_SPGW_DOWNLINK_PDRS);
+    }
+
+    /**
+     * Returns true if the given table entry is a Forwarding Action Rule from the physical fabric pipeline, and
+     * false otherwise.
+     *
+     * @param entry the entry that may or may not be a fabric.p4 FAR
+     * @return true if the entry is a fabric.p4 FAR
+     */
+    public boolean isFabricFar(FlowRule entry) {
+        return entry.table().equals(FABRIC_INGRESS_SPGW_FARS);
+    }
+
+    /**
+     * Returns true if the given table entry is an interface table entry from the fabric.p4 physical pipeline, and
+     * false otherwise.
+     *
+     * @param entry the entry that may or may not be a fabric.p4 UPF interface
+     * @return true if the entry is a fabric.p4 UPF interface
+     */
+    public boolean isFabricInterface(FlowRule entry) {
+        return entry.table().equals(FABRIC_INGRESS_SPGW_INTERFACES);
+    }
+
+
+    /**
+     * Translate a fabric.p4 PDR table entry to a PacketDetectionRule instance for easier handling.
+     *
+     * @param entry the fabric.p4 entry to translate
+     * @return the corresponding PacketDetectionRule
+     * @throws UpfProgrammableException if the entry cannot be translated
+     */
+    public PacketDetectionRule fabricEntryToPdr(FlowRule entry)
+            throws UpfProgrammableException {
+        var pdrBuilder = PacketDetectionRule.builder();
+        Pair<PiCriterion, PiTableAction> matchActionPair = FabricUpfTranslatorUtil.fabricEntryToPiPair(entry);
+        PiCriterion match = matchActionPair.getLeft();
+        PiAction action = (PiAction) matchActionPair.getRight();
+
+        // Grab keys and parameters that are present for all PDRs
+        int globalFarId = FabricUpfTranslatorUtil.getParamInt(action, FAR_ID);
+        UpfRuleIdentifier farId = fabricUpfStore.localFarIdOf(globalFarId);
+
+        PiActionId actionId = action.id();
+        if (actionId.equals(FABRIC_INGRESS_SPGW_LOAD_PDR)) {
+            int schedulingPriority = 0;
+            pdrBuilder.withSchedulingPriority(schedulingPriority);
+        } else if (actionId.equals(FABRIC_INGRESS_SPGW_LOAD_PDR_QOS)) {
+            int queueId = FabricUpfTranslatorUtil.getParamInt(action, QID);
+            String schedulingPriority = fabricUpfStore.schedulingPriorityOf(queueId);
+            if (schedulingPriority == null) {
+                throw new UpfProgrammableException("Undefined Scheduling Priority");
+            }
+            pdrBuilder.withSchedulingPriority(Integer.parseInt(schedulingPriority));
+        } else {
+            throw new UpfProgrammableException("Unknown action ID");
+        }
+        pdrBuilder.withCounterId(FabricUpfTranslatorUtil.getParamInt(action, CTR_ID))
+                .withLocalFarId(farId.getSessionLocalId())
+                .withSessionId(farId.getPfcpSessionId());
+
+        if (FabricUpfTranslatorUtil.fieldIsPresent(match, HDR_TEID)) {
+            // F-TEID is only present for GTP-matching PDRs
+            ImmutableByteSequence teid = FabricUpfTranslatorUtil.getFieldValue(match, HDR_TEID);
+            Ip4Address tunnelDst = FabricUpfTranslatorUtil.getFieldAddress(match, HDR_TUNNEL_IPV4_DST);
+            pdrBuilder.withTeid(teid)
+                    .withTunnelDst(tunnelDst);
+        } else if (FabricUpfTranslatorUtil.fieldIsPresent(match, HDR_UE_ADDR)) {
+            // And UE address is only present for non-GTP-matching PDRs
+            pdrBuilder.withUeAddr(FabricUpfTranslatorUtil.getFieldAddress(match, HDR_UE_ADDR));
+        } else {
+            throw new UpfProgrammableException("Read malformed PDR from dataplane!:" + entry);
+        }
+        return pdrBuilder.build();
+    }
+
+    /**
+     * Translate a fabric.p4 FAR table entry to a ForwardActionRule instance for easier handling.
+     *
+     * @param entry the fabric.p4 entry to translate
+     * @return the corresponding ForwardingActionRule
+     * @throws UpfProgrammableException if the entry cannot be translated
+     */
+    public ForwardingActionRule fabricEntryToFar(FlowRule entry)
+            throws UpfProgrammableException {
+        var farBuilder = ForwardingActionRule.builder();
+        Pair<PiCriterion, PiTableAction> matchActionPair = FabricUpfTranslatorUtil.fabricEntryToPiPair(entry);
+        PiCriterion match = matchActionPair.getLeft();
+        PiAction action = (PiAction) matchActionPair.getRight();
+
+        int globalFarId = FabricUpfTranslatorUtil.getFieldInt(match, HDR_FAR_ID);
+        UpfRuleIdentifier farId = fabricUpfStore.localFarIdOf(globalFarId);
+
+        boolean dropFlag = FabricUpfTranslatorUtil.getParamInt(action, DROP) > 0;
+        boolean notifyFlag = FabricUpfTranslatorUtil.getParamInt(action, NOTIFY_CP) > 0;
+
+        // Match keys
+        farBuilder.withSessionId(farId.getPfcpSessionId())
+                .setFarId(farId.getSessionLocalId());
+
+        // Parameters common to all types of FARs
+        farBuilder.setDropFlag(dropFlag)
+                .setNotifyFlag(notifyFlag);
+
+        PiActionId actionId = action.id();
+
+        if (actionId.equals(FABRIC_INGRESS_SPGW_LOAD_TUNNEL_FAR)
+                || actionId.equals(FABRIC_INGRESS_SPGW_LOAD_DBUF_FAR)) {
+            // Grab parameters specific to encapsulating FARs if they're present
+            Ip4Address tunnelSrc = FabricUpfTranslatorUtil.getParamAddress(action, TUNNEL_SRC_ADDR);
+            Ip4Address tunnelDst = FabricUpfTranslatorUtil.getParamAddress(action, TUNNEL_DST_ADDR);
+            ImmutableByteSequence teid = FabricUpfTranslatorUtil.getParamValue(action, TEID);
+            short tunnelSrcPort = (short) FabricUpfTranslatorUtil.getParamInt(action, TUNNEL_SRC_PORT);
+
+            farBuilder.setBufferFlag(actionId.equals(FABRIC_INGRESS_SPGW_LOAD_DBUF_FAR));
+
+            farBuilder.setTunnel(
+                    GtpTunnel.builder()
+                            .setSrc(tunnelSrc)
+                            .setDst(tunnelDst)
+                            .setTeid(teid)
+                            .setSrcPort(tunnelSrcPort)
+                            .build());
+        }
+        return farBuilder.build();
+    }
+
+    /**
+     * Translate a fabric.p4 interface table entry to a UpfInterface instance for easier handling.
+     *
+     * @param entry the fabric.p4 entry to translate
+     * @return the corresponding UpfInterface
+     * @throws UpfProgrammableException if the entry cannot be translated
+     */
+    public UpfInterface fabricEntryToInterface(FlowRule entry)
+            throws UpfProgrammableException {
+        Pair<PiCriterion, PiTableAction> matchActionPair = FabricUpfTranslatorUtil.fabricEntryToPiPair(entry);
+        PiCriterion match = matchActionPair.getLeft();
+        PiAction action = (PiAction) matchActionPair.getRight();
+
+        var ifaceBuilder = UpfInterface.builder()
+                .setPrefix(FabricUpfTranslatorUtil.getFieldPrefix(match, HDR_IPV4_DST_ADDR));
+
+        int interfaceType = FabricUpfTranslatorUtil.getParamInt(action, SRC_IFACE);
+        if (interfaceType == INTERFACE_ACCESS) {
+            ifaceBuilder.setAccess();
+        } else if (interfaceType == INTERFACE_CORE) {
+            ifaceBuilder.setCore();
+        } else if (interfaceType == INTERFACE_DBUF) {
+            ifaceBuilder.setDbufReceiver();
+        }
+        return ifaceBuilder.build();
+    }
+
+    /**
+     * Translate a ForwardingActionRule to a FlowRule to be inserted into the fabric.p4 pipeline.
+     * A side effect of calling this method is the FAR object's globalFarId is assigned if it was not already.
+     *
+     * @param far      The FAR to be translated
+     * @param deviceId the ID of the device the FlowRule should be installed on
+     * @param appId    the ID of the application that will insert the FlowRule
+     * @param priority the FlowRule's priority
+     * @return the FAR translated to a FlowRule
+     * @throws UpfProgrammableException if the FAR to be translated is malformed
+     */
+    public FlowRule farToFabricEntry(ForwardingActionRule far, DeviceId deviceId, ApplicationId appId, int priority)
+            throws UpfProgrammableException {
+        PiAction action;
+        if (!far.encaps()) {
+            action = PiAction.builder()
+                    .withId(FABRIC_INGRESS_SPGW_LOAD_NORMAL_FAR)
+                    .withParameters(Arrays.asList(
+                            new PiActionParam(DROP, far.drops() ? 1 : 0),
+                            new PiActionParam(NOTIFY_CP, far.notifies() ? 1 : 0)
+                    ))
+                    .build();
+
+        } else {
+            if (far.tunnelSrc() == null || far.tunnelDst() == null
+                    || far.teid() == null || far.tunnel().srcPort() == null) {
+                throw new UpfProgrammableException(
+                        "Not all action parameters present when translating " +
+                                "intermediate encapsulating/buffering FAR to physical FAR!");
+            }
+            // TODO: copy tunnel destination port from logical switch write requests, instead of hardcoding 2152
+            PiActionId actionId = far.buffers() ? FABRIC_INGRESS_SPGW_LOAD_DBUF_FAR :
+                    FABRIC_INGRESS_SPGW_LOAD_TUNNEL_FAR;
+            action = PiAction.builder()
+                    .withId(actionId)
+                    .withParameters(Arrays.asList(
+                            new PiActionParam(DROP, far.drops() ? 1 : 0),
+                            new PiActionParam(NOTIFY_CP, far.notifies() ? 1 : 0),
+                            new PiActionParam(TEID, far.teid()),
+                            new PiActionParam(TUNNEL_SRC_ADDR, far.tunnelSrc().toInt()),
+                            new PiActionParam(TUNNEL_DST_ADDR, far.tunnelDst().toInt()),
+                            new PiActionParam(TUNNEL_SRC_PORT, far.tunnel().srcPort())
+                    ))
+                    .build();
+        }
+        PiCriterion match = PiCriterion.builder()
+                .matchExact(HDR_FAR_ID, fabricUpfStore.globalFarIdOf(far.sessionId(), far.farId()))
+                .build();
+        return DefaultFlowRule.builder()
+                .forDevice(deviceId).fromApp(appId).makePermanent()
+                .forTable(FABRIC_INGRESS_SPGW_FARS)
+                .withSelector(DefaultTrafficSelector.builder().matchPi(match).build())
+                .withTreatment(DefaultTrafficTreatment.builder().piTableAction(action).build())
+                .withPriority(priority)
+                .build();
+    }
+
+    /**
+     * Translate a PacketDetectionRule to a FlowRule to be inserted into the fabric.p4 pipeline.
+     * A side effect of calling this method is the PDR object's globalFarId is assigned if it was not already.
+     *
+     * @param pdr      The PDR to be translated
+     * @param deviceId the ID of the device the FlowRule should be installed on
+     * @param appId    the ID of the application that will insert the FlowRule
+     * @param priority the FlowRule's priority
+     * @return the FAR translated to a FlowRule
+     * @throws UpfProgrammableException if the PDR to be translated is malformed
+     */
+    public FlowRule pdrToFabricEntry(PacketDetectionRule pdr, DeviceId deviceId, ApplicationId appId, int priority)
+            throws UpfProgrammableException {
+        PiCriterion match;
+        PiTableId tableId;
+        PiAction action;
+
+        if (pdr.matchesEncapped()) {
+            match = PiCriterion.builder()
+                    .matchExact(HDR_TEID, pdr.teid().asArray())
+                    .matchExact(HDR_TUNNEL_IPV4_DST, pdr.tunnelDest().toInt())
+                    .build();
+            tableId = FABRIC_INGRESS_SPGW_UPLINK_PDRS;
+        } else if (pdr.matchesUnencapped()) {
+            match = PiCriterion.builder()
+                    .matchExact(HDR_UE_ADDR, pdr.ueAddress().toInt())
+                    .build();
+            tableId = FABRIC_INGRESS_SPGW_DOWNLINK_PDRS;
+        } else {
+            throw new UpfProgrammableException("Flexible PDRs not yet supported! Cannot translate " + pdr.toString());
+        }
+
+        PiAction.Builder builder = PiAction.builder()
+                .withParameters(Arrays.asList(
+                        new PiActionParam(CTR_ID, pdr.counterId()),
+                        new PiActionParam(FAR_ID, fabricUpfStore.globalFarIdOf(pdr.sessionId(), pdr.farId())),
+                        new PiActionParam(NEEDS_GTPU_DECAP, pdr.matchesEncapped() ? 1 : 0)
+                ));
+        if (pdr.hasSchedulingPriority()) {
+            String queueId = fabricUpfStore.queueIdOf(pdr.schedulingPriority());
+            if (queueId == null) {
+                throw new UpfProgrammableException("Udefined Scheduling Priority");
+            }
+            action = builder
+                    .withId(FABRIC_INGRESS_SPGW_LOAD_PDR_QOS)
+                    .withParameter(new PiActionParam(QID, Integer.parseInt(queueId)))
+                    .build();
+        } else {
+            action = builder
+                    .withId(FABRIC_INGRESS_SPGW_LOAD_PDR)
+                    .build();
+        }
+        return DefaultFlowRule.builder()
+                .forDevice(deviceId).fromApp(appId).makePermanent()
+                .forTable(tableId)
+                .withSelector(DefaultTrafficSelector.builder().matchPi(match).build())
+                .withTreatment(DefaultTrafficTreatment.builder().piTableAction(action).build())
+                .withPriority(priority)
+                .build();
+    }
+
+    /**
+     * Translate a UpfInterface to a FlowRule to be inserted into the fabric.p4 pipeline.
+     *
+     * @param upfInterface The interface to be translated
+     * @param deviceId     the ID of the device the FlowRule should be installed on
+     * @param appId        the ID of the application that will insert the FlowRule
+     * @param priority     the FlowRule's priority
+     * @return the UPF interface translated to a FlowRule
+     * @throws UpfProgrammableException if the interface cannot be translated
+     */
+    public FlowRule interfaceToFabricEntry(UpfInterface upfInterface, DeviceId deviceId,
+                                           ApplicationId appId, int priority)
+            throws UpfProgrammableException {
+        int interfaceTypeInt;
+        int gtpuValidity;
+        if (upfInterface.isDbufReceiver()) {
+            interfaceTypeInt = INTERFACE_DBUF;
+            gtpuValidity = 1;
+        } else if (upfInterface.isAccess()) {
+            interfaceTypeInt = INTERFACE_ACCESS;
+            gtpuValidity = 1;
+        } else {
+            interfaceTypeInt = INTERFACE_CORE;
+            gtpuValidity = 0;
+        }
+
+        PiCriterion match = PiCriterion.builder()
+                .matchLpm(HDR_IPV4_DST_ADDR,
+                          upfInterface.prefix().address().toInt(),
+                          upfInterface.prefix().prefixLength())
+                .matchExact(HDR_GTPU_IS_VALID, gtpuValidity)
+                .build();
+        PiAction action = PiAction.builder()
+                .withId(FABRIC_INGRESS_SPGW_LOAD_IFACE)
+                .withParameter(new PiActionParam(SRC_IFACE, interfaceTypeInt))
+                .build();
+        return DefaultFlowRule.builder()
+                .forDevice(deviceId).fromApp(appId).makePermanent()
+                .forTable(FABRIC_INGRESS_SPGW_INTERFACES)
+                .withSelector(DefaultTrafficSelector.builder().matchPi(match).build())
+                .withTreatment(DefaultTrafficTreatment.builder().piTableAction(action).build())
+                .withPriority(priority)
+                .build();
+    }
+
+//    public FlowRule buildGtpuWithPscEncapRule(DeviceId deviceId, ApplicationId appId, int qfi) {
+//        PiAction action = PiAction.builder()
+//                .withId(FABRIC_EGRESS_SPGW_GTPU_WITH_PSC)
+//                .withParameter(new PiActionParam(QFI, qfi))
+//                .build();
+//        // Default entry, no selector.
+//        return DefaultFlowRule.builder()
+//                .forDevice(deviceId).fromApp(appId).makePermanent()
+//                .forTable(FABRIC_EGRESS_SPGW_GTPU_ENCAP)
+//                .withTreatment(DefaultTrafficTreatment.builder().piTableAction(action).build())
+//                .withPriority(0)
+//                .build();
+//    }
+//
+//    public FlowRule buildGtpuOnlyEncapRule(DeviceId deviceId, ApplicationId appId) {
+//        PiAction action = PiAction.builder()
+//                .withId(FABRIC_EGRESS_SPGW_GTPU_ONLY)
+//                .build();
+//        // Default entry, no selector.
+//        return DefaultFlowRule.builder()
+//                .forDevice(deviceId).fromApp(appId).makePermanent()
+//                .forTable(FABRIC_EGRESS_SPGW_GTPU_ENCAP)
+//                .withTreatment(DefaultTrafficTreatment.builder().piTableAction(action).build())
+//                .withPriority(0)
+//                .build();
+//    }
+
+}
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfTranslatorUtil.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfTranslatorUtil.java
new file mode 100644
index 0000000..a931f1f
--- /dev/null
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/FabricUpfTranslatorUtil.java
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2021-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.pipelines.fabric.impl.behaviour.upf;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.onlab.packet.Ip4Address;
+import org.onlab.packet.Ip4Prefix;
+import org.onlab.util.ImmutableByteSequence;
+import org.onosproject.net.behaviour.upf.UpfProgrammableException;
+import org.onosproject.net.flow.FlowRule;
+import org.onosproject.net.flow.criteria.Criterion;
+import org.onosproject.net.flow.criteria.PiCriterion;
+import org.onosproject.net.flow.instructions.Instruction;
+import org.onosproject.net.flow.instructions.PiInstruction;
+import org.onosproject.net.pi.model.PiActionParamId;
+import org.onosproject.net.pi.model.PiMatchFieldId;
+import org.onosproject.net.pi.model.PiMatchType;
+import org.onosproject.net.pi.runtime.PiAction;
+import org.onosproject.net.pi.runtime.PiActionParam;
+import org.onosproject.net.pi.runtime.PiExactFieldMatch;
+import org.onosproject.net.pi.runtime.PiFieldMatch;
+import org.onosproject.net.pi.runtime.PiLpmFieldMatch;
+import org.onosproject.net.pi.runtime.PiRangeFieldMatch;
+import org.onosproject.net.pi.runtime.PiTableAction;
+import org.onosproject.net.pi.runtime.PiTernaryFieldMatch;
+
+import java.util.Optional;
+
+/**
+ * Utility class for manipulation of FlowRules and PiTableEntry objects specific to fabric-tna.
+ */
+final class FabricUpfTranslatorUtil {
+
+    private FabricUpfTranslatorUtil() {
+    }
+
+    static ImmutableByteSequence getFieldValue(PiFieldMatch field, PiMatchFieldId fieldId)
+            throws UpfProgrammableException {
+        if (field == null) {
+            throw new UpfProgrammableException(
+                    String.format("Unable to find field %s where expected!", fieldId.toString()));
+        }
+        if (field.type() == PiMatchType.EXACT) {
+            return ((PiExactFieldMatch) field).value();
+        } else if (field.type() == PiMatchType.LPM) {
+            return ((PiLpmFieldMatch) field).value();
+        } else if (field.type() == PiMatchType.TERNARY) {
+            return ((PiTernaryFieldMatch) field).value();
+        } else if (field.type() == PiMatchType.RANGE) {
+            return ((PiRangeFieldMatch) field).lowValue();
+        } else {
+            throw new UpfProgrammableException(
+                    String.format("Field %s has unknown match type: %s", fieldId.toString(), field.type().toString()));
+        }
+    }
+
+    static ImmutableByteSequence getFieldValue(PiCriterion criterion, PiMatchFieldId fieldId)
+            throws UpfProgrammableException {
+        return getFieldValue(criterion.fieldMatch(fieldId).orElse(null), fieldId);
+    }
+
+    static boolean fieldIsPresent(PiCriterion criterion, PiMatchFieldId fieldId) {
+        return criterion.fieldMatch(fieldId).isPresent();
+    }
+
+    static ImmutableByteSequence getParamValue(PiAction action, PiActionParamId paramId)
+            throws UpfProgrammableException {
+
+        for (PiActionParam param : action.parameters()) {
+            if (param.id().equals(paramId)) {
+                return param.value();
+            }
+        }
+        throw new UpfProgrammableException(
+                String.format("Unable to find parameter %s where expected!", paramId.toString()));
+    }
+
+    static int getFieldInt(PiCriterion criterion, PiMatchFieldId fieldId)
+            throws UpfProgrammableException {
+        return byteSeqToInt(getFieldValue(criterion, fieldId));
+    }
+
+    static int getParamInt(PiAction action, PiActionParamId paramId)
+            throws UpfProgrammableException {
+        return byteSeqToInt(getParamValue(action, paramId));
+    }
+
+    static Ip4Address getParamAddress(PiAction action, PiActionParamId paramId)
+            throws UpfProgrammableException {
+        return Ip4Address.valueOf(getParamValue(action, paramId).asArray());
+    }
+
+    static Ip4Prefix getFieldPrefix(PiCriterion criterion, PiMatchFieldId fieldId) {
+        Optional<PiFieldMatch> optField = criterion.fieldMatch(fieldId);
+        if (optField.isEmpty()) {
+            return null;
+        }
+        PiLpmFieldMatch field = (PiLpmFieldMatch) optField.get();
+        Ip4Address address = Ip4Address.valueOf(field.value().asArray());
+        return Ip4Prefix.valueOf(address, field.prefixLength());
+    }
+
+    static Ip4Address getFieldAddress(PiCriterion criterion, PiMatchFieldId fieldId)
+            throws UpfProgrammableException {
+        return Ip4Address.valueOf(getFieldValue(criterion, fieldId).asArray());
+    }
+
+    static int byteSeqToInt(ImmutableByteSequence sequence) {
+        try {
+            return sequence.fit(32).asReadOnlyBuffer().getInt();
+        } catch (ImmutableByteSequence.ByteSequenceTrimException e) {
+            throw new IllegalArgumentException("Attempted to convert a >4 byte wide sequence to an integer!");
+        }
+    }
+
+    static Pair<PiCriterion, PiTableAction> fabricEntryToPiPair(FlowRule entry) {
+        PiCriterion match = (PiCriterion) entry.selector().getCriterion(Criterion.Type.PROTOCOL_INDEPENDENT);
+        PiTableAction action = null;
+        for (Instruction instruction : entry.treatment().allInstructions()) {
+            if (instruction.type() == Instruction.Type.PROTOCOL_INDEPENDENT) {
+                PiInstruction piInstruction = (PiInstruction) instruction;
+                action = piInstruction.action();
+                break;
+            }
+        }
+        return Pair.of(match, action);
+    }
+}
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/UpfRuleIdentifier.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/UpfRuleIdentifier.java
new file mode 100644
index 0000000..5b6c6b6
--- /dev/null
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/UpfRuleIdentifier.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2021-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.pipelines.fabric.impl.behaviour.upf;
+
+import org.onlab.util.ImmutableByteSequence;
+
+import java.util.Objects;
+
+/**
+ * Wrapper for identifying information of FARs and PDRs.
+ */
+public final class UpfRuleIdentifier {
+    private final int sessionlocalId;
+    private final ImmutableByteSequence pfcpSessionId;
+
+    /**
+     * A PDR or FAR can be globally uniquely identified by the combination of the ID of the PFCP session that
+     * produced it, and the ID that the rule was assigned in that PFCP session.
+     *
+     * @param pfcpSessionId  The PFCP session that produced the rule ID
+     * @param sessionlocalId The rule ID
+     */
+    public UpfRuleIdentifier(ImmutableByteSequence pfcpSessionId, int sessionlocalId) {
+        this.pfcpSessionId = pfcpSessionId;
+        this.sessionlocalId = sessionlocalId;
+    }
+
+    /**
+     * Create an instance of this class from the given PFCP session ID and the session-local Rule ID.
+     *
+     * @param pfcpSessionId  PFCP session ID of the rule to identify
+     * @param sessionlocalId session-local Rule ID of the rule to identify
+     * @return a new rule identifier
+     */
+    public static UpfRuleIdentifier of(ImmutableByteSequence pfcpSessionId, int sessionlocalId) {
+        return new UpfRuleIdentifier(pfcpSessionId, sessionlocalId);
+    }
+
+    /**
+     * Get the PFCP session-local rule ID.
+     *
+     * @return session-local rule ID
+     */
+    public int getSessionLocalId() {
+        return sessionlocalId;
+    }
+
+    /**
+     * Get the PFCP session ID.
+     *
+     * @return PFCP session ID
+     */
+    public ImmutableByteSequence getPfcpSessionId() {
+        return pfcpSessionId;
+    }
+
+    @Override
+    public String toString() {
+        return "RuleIdentifier{" +
+                "sessionlocalId=" + sessionlocalId +
+                ", pfcpSessionId=" + pfcpSessionId +
+                '}';
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == this) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        UpfRuleIdentifier that = (UpfRuleIdentifier) obj;
+        return (this.sessionlocalId == that.sessionlocalId) && (this.pfcpSessionId.equals(that.pfcpSessionId));
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(this.sessionlocalId, this.pfcpSessionId);
+    }
+}
diff --git a/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/package-info.java b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/package-info.java
new file mode 100644
index 0000000..42591786
--- /dev/null
+++ b/pipelines/fabric/impl/src/main/java/org/onosproject/pipelines/fabric/impl/behaviour/upf/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2021-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.
+ */
+
+/**
+ * UPF programmable behaviour implementation for fabric-v1model.
+ */
+package org.onosproject.pipelines.fabric.impl.behaviour.upf;