ONOS-6613 Non-disruptive intent reallocation

Change-Id: I5d051c20402a226ad540b8bc08695b602ff75273
diff --git a/core/api/src/main/java/org/onosproject/net/intent/IntentData.java b/core/api/src/main/java/org/onosproject/net/intent/IntentData.java
index 71c2ebf..bb7a710 100644
--- a/core/api/src/main/java/org/onosproject/net/intent/IntentData.java
+++ b/core/api/src/main/java/org/onosproject/net/intent/IntentData.java
@@ -450,6 +450,13 @@
                 return false;
             }
             // FALLTHROUGH
+        case REALLOCATING:
+            if (currentState == REALLOCATING) {
+                log.trace("{} update not acceptable: no-op REALLOCATING", newData.key());
+                return false;
+            } else if (currentState == INSTALLED) {
+                return true;
+            }
         case INSTALLED:
             if (currentState == INSTALLED) {
                 return false;
diff --git a/core/api/src/main/java/org/onosproject/net/intent/IntentEvent.java b/core/api/src/main/java/org/onosproject/net/intent/IntentEvent.java
index bc30408..47aa94d 100644
--- a/core/api/src/main/java/org/onosproject/net/intent/IntentEvent.java
+++ b/core/api/src/main/java/org/onosproject/net/intent/IntentEvent.java
@@ -63,7 +63,12 @@
         /**
          * Signifies that an intent has been purged from the system.
          */
-        PURGED
+        PURGED,
+
+        /**
+         * Signifies that an intent is being reallocated.
+         */
+        REALLOCATING
     }
 
     /**
@@ -118,6 +123,9 @@
             case INSTALLED:
                 type = Type.INSTALLED;
                 break;
+            case REALLOCATING:
+                type = Type.REALLOCATING;
+                break;
             case WITHDRAW_REQ:
                 type = Type.WITHDRAW_REQ;
                 break;
diff --git a/core/api/src/main/java/org/onosproject/net/intent/IntentState.java b/core/api/src/main/java/org/onosproject/net/intent/IntentState.java
index 88c76e4..2c75e7f 100644
--- a/core/api/src/main/java/org/onosproject/net/intent/IntentState.java
+++ b/core/api/src/main/java/org/onosproject/net/intent/IntentState.java
@@ -16,6 +16,7 @@
 package org.onosproject.net.intent;
 
 import com.google.common.annotations.Beta;
+import org.onosproject.net.intent.constraint.NonDisruptiveConstraint;
 
 /**
  * Representation of the phases an intent may attain during its lifecycle.
@@ -113,5 +114,15 @@
      * in WITHDRAWN or FAILED.
      * </p>
      */
-    PURGE_REQ
+    PURGE_REQ,
+
+    /**
+     * Indicates that the intent is being reallocated in a non-disruptive way {@link NonDisruptiveConstraint}.
+     * This state can be used to inform other applications that the intent is performing
+     * the reallocation. In particular type of networks this operation can require
+     * a long time. After all the reallocation stages are completed, the intent
+     * returns to the {@link #INSTALLED} state. If any of the reallocation stages fails,
+     * the intent is flagged as {@link #FAILED}.
+     */
+    REALLOCATING
 }
diff --git a/core/api/src/main/java/org/onosproject/net/intent/constraint/NonDisruptiveConstraint.java b/core/api/src/main/java/org/onosproject/net/intent/constraint/NonDisruptiveConstraint.java
new file mode 100644
index 0000000..4e5fb6b
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/net/intent/constraint/NonDisruptiveConstraint.java
@@ -0,0 +1,59 @@
+/*
+ * 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.net.intent.constraint;
+
+import org.onosproject.net.intent.ConnectivityIntent;
+import org.onosproject.net.intent.Intent;
+
+/**
+ * Constraint to request a non-disruptive intent reallocation.
+ */
+public final class NonDisruptiveConstraint extends MarkerConstraint {
+
+    private static final NonDisruptiveConstraint NON_DISRUPTIVE_CONSTRAINT = new NonDisruptiveConstraint();
+
+    protected NonDisruptiveConstraint() {
+    }
+
+    /**
+     * Determines whether the intent requires a non-disruptive reallocation.
+     *
+     * @param intent  intent to be inspected
+     * @return        whether the intent has a NonDisruptiveConstraint
+     */
+    public static boolean requireNonDisruptive(Intent intent) {
+        if (intent instanceof ConnectivityIntent) {
+            ConnectivityIntent connectivityIntent = (ConnectivityIntent) intent;
+            return connectivityIntent.constraints().stream()
+                    .anyMatch(p -> p instanceof NonDisruptiveConstraint);
+        }
+        return false;
+    }
+
+    /**
+     * Returns the nonDisruptiveConstraint.
+     *
+     * @return non-disruptive constraint
+     */
+    public static NonDisruptiveConstraint nonDisruptive() {
+        return NON_DISRUPTIVE_CONSTRAINT;
+    }
+
+    @Override
+    public String toString() {
+        return "Non-disruptive reallocation required";
+    }
+}
diff --git a/core/common/src/main/java/org/onosproject/codec/impl/DecodeConstraintCodecHelper.java b/core/common/src/main/java/org/onosproject/codec/impl/DecodeConstraintCodecHelper.java
index 3b10907..22c50b1 100644
--- a/core/common/src/main/java/org/onosproject/codec/impl/DecodeConstraintCodecHelper.java
+++ b/core/common/src/main/java/org/onosproject/codec/impl/DecodeConstraintCodecHelper.java
@@ -29,6 +29,7 @@
 import org.onosproject.net.intent.constraint.DomainConstraint;
 import org.onosproject.net.intent.constraint.LatencyConstraint;
 import org.onosproject.net.intent.constraint.LinkTypeConstraint;
+import org.onosproject.net.intent.constraint.NonDisruptiveConstraint;
 import org.onosproject.net.intent.constraint.ObstacleConstraint;
 import org.onosproject.net.intent.constraint.WaypointConstraint;
 
@@ -36,6 +37,7 @@
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 import static org.onlab.util.Tools.nullIsIllegal;
+import static org.onosproject.net.intent.constraint.NonDisruptiveConstraint.nonDisruptive;
 
 /**
  * Constraint JSON decoder.
@@ -183,6 +185,15 @@
     }
 
     /**
+     * Decodes a non-disruptive reallocation constraint.
+     *
+     * @return non-disruptive reallocation constraint object.
+     */
+    private Constraint decodeNonDisruptiveConstraint() {
+        return nonDisruptive();
+    }
+
+    /**
      * Decodes the given constraint.
      *
      * @return constraint object.
@@ -208,6 +219,8 @@
             return decodeAsymmetricPathConstraint();
         } else if (type.equals(DomainConstraint.class.getSimpleName())) {
             return decodeDomainConstraint();
+        } else if (type.equals(NonDisruptiveConstraint.class.getSimpleName())) {
+            return decodeNonDisruptiveConstraint();
         }
         throw new IllegalArgumentException("Instruction type "
                 + type + " is not supported");
diff --git a/core/common/src/main/java/org/onosproject/codec/impl/IntentCodec.java b/core/common/src/main/java/org/onosproject/codec/impl/IntentCodec.java
index 69d10bc..ffe281b 100644
--- a/core/common/src/main/java/org/onosproject/codec/impl/IntentCodec.java
+++ b/core/common/src/main/java/org/onosproject/codec/impl/IntentCodec.java
@@ -15,22 +15,23 @@
  */
 package org.onosproject.codec.impl;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.net.UrlEscapers;
 import org.onosproject.codec.CodecContext;
 import org.onosproject.codec.JsonCodec;
 import org.onosproject.core.CoreService;
 import org.onosproject.net.Link;
 import org.onosproject.net.ResourceGroup;
-import org.onosproject.net.intent.PointToPointIntent;
+import org.onosproject.net.intent.HostToHostIntent;
 import org.onosproject.net.intent.Intent;
 import org.onosproject.net.intent.IntentService;
 import org.onosproject.net.intent.IntentState;
-import org.onosproject.net.intent.HostToHostIntent;
+import org.onosproject.net.intent.Key;
+import org.onosproject.net.intent.PointToPointIntent;
 import org.onosproject.net.intent.SinglePointToMultiPointIntent;
 import org.onosproject.net.intent.MultiPointToSinglePointIntent;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.google.common.net.UrlEscapers;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.onlab.util.Tools.nullIsIllegal;
@@ -43,6 +44,7 @@
 
     protected static final String TYPE = "type";
     protected static final String ID = "id";
+    protected static final String KEY = "key";
     protected static final String APP_ID = "appId";
     protected static final String STATE = "state";
     protected static final String PRIORITY = "priority";
@@ -60,6 +62,7 @@
         final ObjectNode result = context.mapper().createObjectNode()
                 .put(TYPE, intent.getClass().getSimpleName())
                 .put(ID, intent.id().toString())
+                .put(KEY, intent.key().toString())
                 .put(APP_ID, UrlEscapers.urlPathSegmentEscaper()
                         .escape(intent.appId().name()));
         if (intent.resourceGroup() != null) {
@@ -127,6 +130,19 @@
             builder.priority(priorityJson.asInt());
         }
 
+        JsonNode keyJson = json.get(IntentCodec.KEY);
+        if (keyJson != null) {
+            String keyString = keyJson.asText();
+            if (keyString.startsWith("0x")) {
+                // The intent uses a LongKey
+                keyString = keyString.replaceFirst("0x", "");
+                builder.key(Key.of(Long.parseLong(keyString, 16), service.getAppId(appId)));
+            } else {
+                // The intent uses a StringKey
+                builder.key(Key.of(keyString, service.getAppId(appId)));
+            }
+        }
+
         JsonNode resourceGroup = json.get(IntentCodec.RESOURCE_GROUP);
         if (resourceGroup != null) {
             String resourceGroupId = resourceGroup.asText();
diff --git a/core/net/src/main/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstaller.java b/core/net/src/main/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstaller.java
index 8217255..d96de2f 100644
--- a/core/net/src/main/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstaller.java
+++ b/core/net/src/main/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstaller.java
@@ -16,34 +16,57 @@
 
 package org.onosproject.net.intent.impl.installer;
 
+import com.google.common.collect.Lists;
 import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Modified;
+import org.apache.felix.scr.annotations.Property;
 import org.apache.felix.scr.annotations.Reference;
 import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.onlab.util.Tools;
+import org.onosproject.cfg.ComponentConfigService;
+import org.onosproject.core.DefaultApplicationId;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.Link;
+import org.onosproject.net.NetworkResource;
+import org.onosproject.net.flow.DefaultFlowRule;
 import org.onosproject.net.flow.FlowRule;
 import org.onosproject.net.flow.FlowRuleOperations;
 import org.onosproject.net.flow.FlowRuleOperationsContext;
 import org.onosproject.net.flow.FlowRuleService;
 import org.onosproject.net.intent.FlowRuleIntent;
-import org.onosproject.net.intent.IntentInstallCoordinator;
+import org.onosproject.net.intent.Intent;
 import org.onosproject.net.intent.IntentData;
 import org.onosproject.net.intent.IntentExtensionService;
-import org.onosproject.net.intent.IntentOperationContext;
+import org.onosproject.net.intent.IntentInstallCoordinator;
 import org.onosproject.net.intent.IntentInstaller;
-import org.onosproject.net.intent.impl.IntentManager;
+import org.onosproject.net.intent.IntentOperationContext;
+import org.onosproject.net.intent.IntentStore;
 import org.onosproject.net.intent.ObjectiveTrackerService;
+import org.onosproject.net.intent.impl.IntentManager;
+import org.osgi.service.component.ComponentContext;
 import org.slf4j.Logger;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
+import static com.google.common.base.Strings.isNullOrEmpty;
+import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
+import static org.onlab.util.Tools.groupedThreads;
 import static org.onosproject.net.intent.IntentInstaller.Direction.ADD;
 import static org.onosproject.net.intent.IntentInstaller.Direction.REMOVE;
+import static org.onosproject.net.intent.IntentState.INSTALLED;
+import static org.onosproject.net.intent.IntentState.REALLOCATING;
+import static org.onosproject.net.intent.constraint.NonDisruptiveConstraint.requireNonDisruptive;
 import static org.slf4j.LoggerFactory.getLogger;
 
 /**
@@ -63,23 +86,71 @@
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected FlowRuleService flowRuleService;
 
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ComponentConfigService configService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected IntentStore store;
+
+    private ScheduledExecutorService nonDisruptiveIntentInstaller;
+
+    private static final int DEFAULT_NON_DISRUPTIVE_INSTALLATION_WAITING_TIME = 1;
+    @Property(name = "nonDisruptiveInstallationWaitingTime",
+            intValue = DEFAULT_NON_DISRUPTIVE_INSTALLATION_WAITING_TIME,
+            label = "Number of seconds to wait during the non-disruptive installation phases")
+    private int nonDisruptiveInstallationWaitingTime = DEFAULT_NON_DISRUPTIVE_INSTALLATION_WAITING_TIME;
+
+    protected final Logger log = getLogger(IntentManager.class);
+
+    private boolean isReallocationStageFailed = false;
+
+    private static final LinkComparator LINK_COMPARATOR = new LinkComparator();
+
     @Activate
     public void activate() {
         intentExtensionService.registerInstaller(FlowRuleIntent.class, this);
+        nonDisruptiveIntentInstaller =
+                newSingleThreadScheduledExecutor(groupedThreads("onos/intent", "non-disruptive-installer", log));
+        configService.registerProperties(getClass());
     }
 
     @Deactivate
     public void deactivated() {
         intentExtensionService.unregisterInstaller(FlowRuleIntent.class);
+        configService.unregisterProperties(getClass(), false);
     }
 
-    protected final Logger log = getLogger(IntentManager.class);
+    @Modified
+    public void modified(ComponentContext context) {
+
+        if (context == null) {
+            nonDisruptiveInstallationWaitingTime = DEFAULT_NON_DISRUPTIVE_INSTALLATION_WAITING_TIME;
+            log.info("Restored default installation time for non-disruptive reallocation (1 sec.)");
+            return;
+        }
+
+        String s = Tools.get(context.getProperties(), "nonDisruptiveInstallationWaitingTime");
+        int nonDisruptiveTime = isNullOrEmpty(s) ? nonDisruptiveInstallationWaitingTime : Integer.parseInt(s);
+        if (nonDisruptiveTime != nonDisruptiveInstallationWaitingTime) {
+            nonDisruptiveInstallationWaitingTime = nonDisruptiveTime;
+            log.info("Reconfigured non-disruptive reallocation with installation delay {} sec.",
+                     nonDisruptiveInstallationWaitingTime);
+        }
+    }
 
     @Override
     public void apply(IntentOperationContext<FlowRuleIntent> context) {
         Optional<IntentData> toUninstall = context.toUninstall();
         Optional<IntentData> toInstall = context.toInstall();
 
+        if (toInstall.isPresent() && toUninstall.isPresent()) {
+            Intent intentToInstall = toInstall.get().intent();
+            if (requireNonDisruptive(intentToInstall) && INSTALLED.equals(toUninstall.get().state())) {
+                reallocate(context);
+                return;
+            }
+        }
+
         if (!toInstall.isPresent() && !toUninstall.isPresent()) {
             // Nothing to do.
             intentInstallCoordinator.intentInstallSuccess(context);
@@ -176,6 +247,307 @@
         flowRuleService.apply(operations);
     }
 
+    private void reallocate(IntentOperationContext<FlowRuleIntent> context) {
+
+        Optional<IntentData> toUninstall = context.toUninstall();
+        Optional<IntentData> toInstall = context.toInstall();
+
+        //TODO: Update the Intent store with this information
+        toInstall.get().setState(REALLOCATING);
+
+        store.write(toInstall.get());
+
+        List<FlowRuleIntent> uninstallIntents = Lists.newArrayList(context.intentsToUninstall());
+        List<FlowRuleIntent> installIntents = Lists.newArrayList(context.intentsToInstall());
+        FlowRuleOperations.Builder firstStageOperationsBuilder = FlowRuleOperations.builder();
+        List<FlowRule> secondStageFlowRules = Lists.newArrayList();
+        FlowRuleOperations.Builder thirdStageOperationsBuilder = FlowRuleOperations.builder();
+        FlowRuleOperations.Builder finalStageOperationsBuilder = FlowRuleOperations.builder();
+
+        prepareReallocation(uninstallIntents, installIntents,
+                            firstStageOperationsBuilder, secondStageFlowRules,
+                            thirdStageOperationsBuilder, finalStageOperationsBuilder);
+
+        trackIntentResources(toUninstall.get(), uninstallIntents, REMOVE);
+        trackIntentResources(toInstall.get(), installIntents, ADD);
+
+        CountDownLatch stageCompleteLatch = new CountDownLatch(1);
+
+        FlowRuleOperations firstStageOperations = firstStageOperationsBuilder
+                .build(new StageOperation(context, stageCompleteLatch));
+
+        flowRuleService.apply(firstStageOperations);
+
+        try {
+            stageCompleteLatch.await(nonDisruptiveInstallationWaitingTime, TimeUnit.SECONDS);
+            if (isReallocationStageFailed) {
+                log.error("Reallocation FAILED in stage one: the following FlowRuleOperations are not executed {}",
+                          firstStageOperations);
+                return;
+            } else {
+                log.debug("Reallocation stage one completed");
+            }
+        } catch (Exception e) {
+            log.warn("Latch exception in the reallocation stage one");
+        }
+
+        for (FlowRule flowRule : secondStageFlowRules) {
+            stageCompleteLatch = new CountDownLatch(1);
+            FlowRuleOperations operations = FlowRuleOperations.builder()
+                    .newStage()
+                    .remove(flowRule)
+                    .build(new StageOperation(context, stageCompleteLatch));
+            nonDisruptiveIntentInstaller.schedule(new NonDisruptiveInstallation(operations),
+                                                  nonDisruptiveInstallationWaitingTime,
+                                                  TimeUnit.SECONDS);
+            try {
+                stageCompleteLatch.await(nonDisruptiveInstallationWaitingTime, TimeUnit.SECONDS);
+                if (isReallocationStageFailed) {
+                    log.error("Reallocation FAILED in stage two: " +
+                                      "the following FlowRuleOperations are not executed {}",
+                              operations);
+                    return;
+                } else {
+                    log.debug("Reallocation stage two completed");
+                }
+            } catch (Exception e) {
+                log.warn("Latch exception in the reallocation stage two");
+            }
+        }
+
+        stageCompleteLatch = new CountDownLatch(1);
+        FlowRuleOperations thirdStageOperations = thirdStageOperationsBuilder
+                .build(new StageOperation(context, stageCompleteLatch));
+
+        nonDisruptiveIntentInstaller.schedule(new NonDisruptiveInstallation(thirdStageOperations),
+                                              nonDisruptiveInstallationWaitingTime,
+                                              TimeUnit.SECONDS);
+        try {
+            stageCompleteLatch.await(nonDisruptiveInstallationWaitingTime, TimeUnit.SECONDS);
+            if (isReallocationStageFailed) {
+                log.error("Reallocation FAILED in stage three: " +
+                                  "the following FlowRuleOperations are not executed {}",
+                          thirdStageOperations);
+                return;
+            } else {
+                log.debug("Reallocation stage three completed");
+            }
+        } catch (Exception e) {
+            log.warn("Latch exception in the reallocation stage three");
+        }
+
+        FlowRuleOperationsContext flowRuleOperationsContext = new FlowRuleOperationsContext() {
+            @Override
+            public void onSuccess(FlowRuleOperations ops) {
+                intentInstallCoordinator.intentInstallSuccess(context);
+                log.info("Non-disruptive reallocation completed for intent {}", toInstall.get().key());
+            }
+
+            @Override
+            public void onError(FlowRuleOperations ops) {
+                intentInstallCoordinator.intentInstallFailed(context);
+            }
+        };
+
+        FlowRuleOperations finalStageOperations = finalStageOperationsBuilder.build(flowRuleOperationsContext);
+        flowRuleService.apply(finalStageOperations);
+    }
+
+    /**
+     * This method prepares the {@link FlowRule} required for every reallocation stage.
+     *     <p>Stage 1: the FlowRules of the new path are installed,
+     *     with a lower priority only on the devices shared with the old path;</p>
+     *     <p>Stage 2: the FlowRules of the old path are removed from the ingress to the egress points,
+     *     only in the shared devices;</p>
+     *     <p>Stage 3: the FlowRules with a lower priority are restored to the original one;</p>
+     *     <p>Stage 4: the remaining FlowRules of the old path are deleted.</p>
+     *
+     * @param uninstallIntents the previous FlowRuleIntent
+     * @param installIntents the new FlowRuleIntent to be installed
+     * @param firstStageBuilder the first stage operation builder
+     * @param secondStageFlowRules the second stage FlowRules
+     * @param thirdStageBuilder the third stage operation builder
+     * @param finalStageBuilder the last stage operation builder
+     */
+    private void prepareReallocation(List<FlowRuleIntent> uninstallIntents, List<FlowRuleIntent> installIntents,
+                                     FlowRuleOperations.Builder firstStageBuilder,
+                                     List<FlowRule> secondStageFlowRules,
+                                     FlowRuleOperations.Builder thirdStageBuilder,
+                                     FlowRuleOperations.Builder finalStageBuilder) {
+
+
+        // Filter out same intents and intents with same flow rules
+        installIntents.forEach(installIntent -> {
+            uninstallIntents.forEach(uninstallIntent -> {
+
+                List<FlowRule> uninstallFlowRules = Lists.newArrayList(uninstallIntent.flowRules());
+                List<FlowRule> installFlowRules = Lists.newArrayList(installIntent.flowRules());
+
+                List<FlowRule> secondStageRules = Lists.newArrayList();
+                List<FlowRule> thirdStageRules = Lists.newArrayList();
+
+                List<DeviceId> orderedDeviceList = createIngressToEgressDeviceList(installIntent.resources());
+
+                uninstallIntent.flowRules().forEach(flowRuleToUnistall -> {
+                    installIntent.flowRules().forEach(flowRuleToInstall -> {
+
+                        if (flowRuleToInstall.exactMatch(flowRuleToUnistall)) {
+                            //The FlowRules are in common (i.e., we are sharing the path)
+                            uninstallFlowRules.remove(flowRuleToInstall);
+                            installFlowRules.remove(flowRuleToInstall);
+                        } else if (flowRuleToInstall.deviceId().equals(flowRuleToUnistall.deviceId())) {
+                            //FlowRules that have a device in common but
+                            // different treatment/selector (i.e., overlapping path)
+                            FlowRule flowRuleWithLowerPriority = DefaultFlowRule.builder()
+                                    .withPriority(flowRuleToInstall.priority() - 1)
+                                    .withSelector(flowRuleToInstall.selector())
+                                    .forDevice(flowRuleToInstall.deviceId())
+                                    .makePermanent()
+                                    .withTreatment(flowRuleToInstall.treatment())
+                                    .fromApp(new DefaultApplicationId(flowRuleToInstall.appId(),
+                                                                      "org.onosproject.net.intent"))
+                                    .build();
+
+                            //Update the FlowRule to be installed with one with a lower priority
+                            installFlowRules.remove(flowRuleToInstall);
+                            installFlowRules.add(flowRuleWithLowerPriority);
+
+                            //Add the FlowRule to be uninstalled to the second stage of non-disruptive update
+                            secondStageRules.add(flowRuleToUnistall);
+                            uninstallFlowRules.remove(flowRuleToUnistall);
+
+                            thirdStageRules.add(flowRuleToInstall);
+                            uninstallFlowRules.add(flowRuleWithLowerPriority);
+                        }
+                    });
+                });
+
+                firstStageBuilder.newStage();
+                installFlowRules.forEach(firstStageBuilder::add);
+
+                Collections.sort(secondStageRules, new SecondStageComparator(orderedDeviceList));
+                secondStageFlowRules.addAll(secondStageRules);
+
+                thirdStageBuilder.newStage();
+                thirdStageRules.forEach(thirdStageBuilder::add);
+
+                finalStageBuilder.newStage();
+                uninstallFlowRules.forEach(finalStageBuilder::remove);
+            });
+        });
+
+    }
+
+    private class StageOperation implements FlowRuleOperationsContext {
+
+        private IntentOperationContext<FlowRuleIntent> context;
+        private CountDownLatch stageCompleteLatch;
+
+        public StageOperation(IntentOperationContext<FlowRuleIntent> context, CountDownLatch stageCompleteLatch) {
+            this.context = context;
+            this.stageCompleteLatch = stageCompleteLatch;
+            isReallocationStageFailed = false;
+        }
+
+        @Override
+        public void onSuccess(FlowRuleOperations ops) {
+            stageCompleteLatch.countDown();
+            log.debug("FlowRuleOperations correctly completed");
+        }
+
+        @Override
+        public void onError(FlowRuleOperations ops) {
+            intentInstallCoordinator.intentInstallFailed(context);
+            isReallocationStageFailed = true;
+            stageCompleteLatch.countDown();
+            log.debug("Installation error for {}", ops);
+        }
+    }
+
+    private final class SecondStageComparator implements Comparator<FlowRule> {
+
+        private List<DeviceId> deviceIds;
+
+        private SecondStageComparator(List<DeviceId> deviceIds) {
+            this.deviceIds = deviceIds;
+        }
+
+        @Override
+        public int compare(FlowRule o1, FlowRule o2) {
+            Integer index1 = deviceIds.indexOf(o1.deviceId());
+            Integer index2 = deviceIds.indexOf(o2.deviceId());
+            return index1.compareTo(index2);
+        }
+    }
+
+    /**
+     * Create a list of devices ordered from the ingress to the egress of a path.
+     * @param resources the resources of the intent
+     * @return a list of devices
+     */
+    private List<DeviceId> createIngressToEgressDeviceList(Collection<NetworkResource> resources) {
+        List<DeviceId> deviceIds = Lists.newArrayList();
+        List<Link> links = Lists.newArrayList();
+
+        for (NetworkResource resource : resources) {
+            if (resource instanceof Link) {
+                Link linkToAdd = (Link) resource;
+                if (linkToAdd.type() != Link.Type.EDGE) {
+                    links.add(linkToAdd);
+                }
+            }
+        }
+
+        Collections.sort(links, LINK_COMPARATOR);
+
+        int i = 0;
+        for (Link orderedLink : links) {
+            deviceIds.add(orderedLink.src().deviceId());
+            if (i == resources.size() - 1) {
+                deviceIds.add(orderedLink.dst().deviceId());
+            }
+            i++;
+        }
+
+        return deviceIds;
+    }
+
+    /**
+     * Compares two links in order to find which one is before or after the other.
+     */
+    private static class LinkComparator implements Comparator<Link> {
+
+        @Override
+        public int compare(Link l1, Link l2) {
+
+            //l1 is before l2
+            if (l1.dst().deviceId() == l2.src().deviceId()) {
+                return -1;
+            }
+
+            //l1 is after l2
+            if (l1.src().deviceId() == l2.dst().deviceId()) {
+                return 1;
+            }
+
+            //l2 and l1 are not connected to a common device
+            return 0;
+        }
+    }
+
+    private final class NonDisruptiveInstallation implements Runnable {
+
+        private FlowRuleOperations op;
+
+        private NonDisruptiveInstallation(FlowRuleOperations op) {
+            this.op = op;
+        }
+        @Override
+        public void run() {
+            flowRuleService.apply(this.op);
+        }
+    }
+
     /**
      * Track or un-track network resource of a Intent and it's installable
      * Intents.
diff --git a/core/net/src/test/java/org/onosproject/net/intent/impl/installer/AbstractIntentInstallerTest.java b/core/net/src/test/java/org/onosproject/net/intent/impl/installer/AbstractIntentInstallerTest.java
index b4e51aa..a439103 100644
--- a/core/net/src/test/java/org/onosproject/net/intent/impl/installer/AbstractIntentInstallerTest.java
+++ b/core/net/src/test/java/org/onosproject/net/intent/impl/installer/AbstractIntentInstallerTest.java
@@ -16,24 +16,31 @@
 
 package org.onosproject.net.intent.impl.installer;
 
+import com.google.common.collect.ImmutableList;
 import org.onosproject.TestApplicationId;
 import org.onosproject.core.ApplicationId;
 import org.onosproject.net.ConnectPoint;
 import org.onosproject.net.FilteredConnectPoint;
+import org.onosproject.net.Link;
 import org.onosproject.net.ResourceGroup;
 import org.onosproject.net.flow.DefaultTrafficSelector;
 import org.onosproject.net.flow.DefaultTrafficTreatment;
 import org.onosproject.net.flow.TrafficSelector;
 import org.onosproject.net.flow.TrafficTreatment;
 import org.onosproject.net.intent.AbstractIntentTest;
+import org.onosproject.net.intent.Constraint;
 import org.onosproject.net.intent.IntentExtensionService;
 import org.onosproject.net.intent.IntentInstallCoordinator;
 import org.onosproject.net.intent.IntentOperationContext;
 import org.onosproject.net.intent.Key;
-import org.onosproject.net.intent.PointToPointIntent;
 import org.onosproject.net.intent.ObjectiveTrackerService;
+import org.onosproject.net.intent.PointToPointIntent;
+
+import java.util.List;
 
 import static org.easymock.EasyMock.createMock;
+import static org.onosproject.net.NetTestTools.link;
+import static org.onosproject.net.intent.constraint.NonDisruptiveConstraint.nonDisruptive;
 
 /**
  * Abstract class to hold the common variables and pieces of code for Intent
@@ -41,9 +48,26 @@
  */
 public class AbstractIntentInstallerTest extends AbstractIntentTest {
     protected static final ApplicationId APP_ID = TestApplicationId.create("IntentInstallerTest");
+
     protected static final ConnectPoint CP1 = ConnectPoint.deviceConnectPoint("s1/1");
     protected static final ConnectPoint CP2 = ConnectPoint.deviceConnectPoint("s1/2");
     protected static final ConnectPoint CP3 = ConnectPoint.deviceConnectPoint("s1/3");
+
+    protected static final ConnectPoint CP2_1 = ConnectPoint.deviceConnectPoint("s2/1");
+    protected static final ConnectPoint CP2_2 = ConnectPoint.deviceConnectPoint("s2/2");
+
+    protected static final ConnectPoint CP3_1 = ConnectPoint.deviceConnectPoint("s3/1");
+    protected static final ConnectPoint CP3_2 = ConnectPoint.deviceConnectPoint("s3/2");
+
+    protected static final ConnectPoint CP4_1 = ConnectPoint.deviceConnectPoint("s4/1");
+    protected static final ConnectPoint CP4_2 = ConnectPoint.deviceConnectPoint("s4/2");
+    protected static final ConnectPoint CP4_3 = ConnectPoint.deviceConnectPoint("s4/3");
+
+    protected static final Link S1_S2 = link(CP2, CP2_1);
+    protected static final Link S2_S4 = link(CP2_2, CP4_2);
+    protected static final Link S1_S3 = link(CP3, CP3_1);
+    protected static final Link S3_S4 = link(CP3_2, CP4_3);
+
     protected static final Key KEY1 = Key.of("test intent 1", APP_ID);
     protected static final ResourceGroup RG1 = ResourceGroup.of("test resource group 1");
     protected static final int DEFAULT_PRIORITY = 30000;
@@ -84,6 +108,33 @@
     }
 
     /**
+     * Creates point to point Intent for testing non-disruptive reallocation.
+     *
+     * @return the point to point Intent
+     */
+    public PointToPointIntent createP2PIntentNonDisruptive() {
+        PointToPointIntent intent;
+        TrafficSelector selector = DefaultTrafficSelector.emptySelector();
+        TrafficTreatment treatment = DefaultTrafficTreatment.emptyTreatment();
+
+        FilteredConnectPoint ingress = new FilteredConnectPoint(CP1);
+        FilteredConnectPoint egress = new FilteredConnectPoint(CP4_1);
+
+        List<Constraint> constraints = ImmutableList.of(nonDisruptive());
+
+        intent = PointToPointIntent.builder()
+                .selector(selector)
+                .treatment(treatment)
+                .filteredIngressPoint(ingress)
+                .filteredEgressPoint(egress)
+                .constraints(constraints)
+                .appId(APP_ID)
+                .build();
+
+        return intent;
+    }
+
+    /**
      * The Intent install coordinator for test.
      * Records success and fail context.
      */
diff --git a/core/net/src/test/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstallerTest.java b/core/net/src/test/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstallerTest.java
index f979f27..fdef627 100644
--- a/core/net/src/test/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstallerTest.java
+++ b/core/net/src/test/java/org/onosproject/net/intent/impl/installer/FlowRuleIntentInstallerTest.java
@@ -18,11 +18,14 @@
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.onlab.packet.VlanId;
+import org.onosproject.cfg.ComponentConfigService;
+import org.onosproject.net.ConnectPoint;
 import org.onosproject.net.NetworkResource;
 import org.onosproject.net.flow.DefaultFlowRule;
 import org.onosproject.net.flow.DefaultTrafficSelector;
@@ -40,13 +43,17 @@
 import org.onosproject.net.intent.IntentState;
 import org.onosproject.net.intent.PathIntent;
 import org.onosproject.store.service.WallClockTimestamp;
+import org.onosproject.store.trivial.SimpleIntentStore;
 
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import static org.junit.Assert.*;
+import static org.easymock.EasyMock.mock;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests for flow rule Intent installer.
@@ -54,17 +61,21 @@
 public class FlowRuleIntentInstallerTest extends AbstractIntentInstallerTest {
 
     private TestFlowRuleService flowRuleService;
+    private TestFlowRuleServiceNonDisruptive flowRuleServiceNonDisruptive;
     private FlowRuleIntentInstaller installer;
 
     @Before
     public void setup() {
         super.setup();
         flowRuleService = new TestFlowRuleService();
+        flowRuleServiceNonDisruptive = new TestFlowRuleServiceNonDisruptive();
         installer = new FlowRuleIntentInstaller();
         installer.flowRuleService = flowRuleService;
+        installer.store = new SimpleIntentStore();
         installer.intentExtensionService = intentExtensionService;
         installer.intentInstallCoordinator = intentInstallCoordinator;
         installer.trackerService = trackerService;
+        installer.configService = mock(ComponentConfigService.class);
 
         installer.activate();
     }
@@ -328,6 +339,109 @@
     }
 
     /**
+     * Testing the non-disruptive reallocation.
+     */
+    @Test
+    public void testUninstallAndInstallNonDisruptive() throws InterruptedException {
+
+        installer.flowRuleService = flowRuleServiceNonDisruptive;
+
+        List<Intent> intentsToInstall = createAnotherFlowRuleIntentsNonDisruptive();
+        List<Intent> intentsToUninstall = createFlowRuleIntentsNonDisruptive();
+
+        IntentData toInstall = new IntentData(createP2PIntentNonDisruptive(),
+                                              IntentState.INSTALLING,
+                                              new WallClockTimestamp());
+        toInstall = new IntentData(toInstall, intentsToInstall);
+        IntentData toUninstall = new IntentData(createP2PIntentNonDisruptive(),
+                                                IntentState.INSTALLED,
+                                                new WallClockTimestamp());
+        toUninstall = new IntentData(toUninstall, intentsToUninstall);
+
+        IntentOperationContext<FlowRuleIntent> operationContext;
+        IntentInstallationContext context = new IntentInstallationContext(toUninstall, toInstall);
+        operationContext = new IntentOperationContext(intentsToUninstall, intentsToInstall, context);
+
+        installer.apply(operationContext);
+
+        //A single FlowRule is evaluated for every non-disruptive stage
+        TrafficSelector selector = DefaultTrafficSelector.builder()
+                .matchInPhyPort(CP1.port())
+                .build();
+        TrafficTreatment treatment = DefaultTrafficTreatment.builder()
+                .setOutput(CP3.port())
+                .build();
+
+        FlowRule firstStageInstalledRule = DefaultFlowRule.builder()
+                .forDevice(CP1.deviceId())
+                .withSelector(selector)
+                .withTreatment(treatment)
+                .fromApp(APP_ID)
+                .withPriority(DEFAULT_PRIORITY - 1)
+                .makePermanent()
+                .build();
+
+        assertTrue(flowRuleServiceNonDisruptive.flowRulesAdd.contains(firstStageInstalledRule));
+
+        selector = DefaultTrafficSelector.builder()
+                .matchInPhyPort(CP4_2.port())
+                .build();
+        treatment = DefaultTrafficTreatment.builder()
+                .setOutput(CP4_1.port())
+                .build();
+
+        FlowRule secondStageUninstalledRule = DefaultFlowRule.builder()
+                .forDevice(CP4_1.deviceId())
+                .withSelector(selector)
+                .withTreatment(treatment)
+                .fromApp(APP_ID)
+                .withPriority(DEFAULT_PRIORITY)
+                .makePermanent()
+                .build();
+
+        assertTrue(flowRuleServiceNonDisruptive.flowRulesRemove.contains(secondStageUninstalledRule));
+
+        selector = DefaultTrafficSelector.builder()
+                .matchInPhyPort(CP4_3.port())
+                .build();
+        treatment = DefaultTrafficTreatment.builder()
+                .setOutput(CP4_1.port())
+                .build();
+
+        FlowRule thirdStageInstalledRule = DefaultFlowRule.builder()
+                .forDevice(CP4_1.deviceId())
+                .withSelector(selector)
+                .withTreatment(treatment)
+                .fromApp(APP_ID)
+                .withPriority(DEFAULT_PRIORITY)
+                .makePermanent()
+                .build();
+
+        assertTrue(flowRuleServiceNonDisruptive.flowRulesAdd.contains(thirdStageInstalledRule));
+
+        selector = DefaultTrafficSelector.builder()
+                .matchInPhyPort(CP2_1.port())
+                .build();
+        treatment = DefaultTrafficTreatment.builder()
+                .setOutput(CP2_2.port())
+                .build();
+
+        FlowRule lastStageUninstalledRule = DefaultFlowRule.builder()
+                .forDevice(CP2_1.deviceId())
+                .withSelector(selector)
+                .withTreatment(treatment)
+                .fromApp(APP_ID)
+                .withPriority(DEFAULT_PRIORITY)
+                .makePermanent()
+                .build();
+
+        assertTrue(flowRuleServiceNonDisruptive.flowRulesRemove.contains(lastStageUninstalledRule));
+
+        IntentOperationContext successContext = intentInstallCoordinator.successContext;
+        assertEquals(successContext, operationContext);
+    }
+
+    /**
      * Generates FlowRuleIntents for test.
      *
      * @return the FlowRuleIntents for test
@@ -442,6 +556,107 @@
     }
 
     /**
+     * Generates FlowRuleIntents for testing non-disruptive reallocation.
+     *
+     * @return the FlowRuleIntents for test
+     */
+    public List<Intent> createFlowRuleIntentsNonDisruptive() {
+
+        Map<ConnectPoint, ConnectPoint> portsAssociation = Maps.newHashMap();
+        portsAssociation.put(CP1, CP2);
+        portsAssociation.put(CP2_1, CP2_2);
+        portsAssociation.put(CP4_2, CP4_1);
+
+        List<FlowRule> flowRules = Lists.newArrayList();
+
+        for (ConnectPoint srcPoint : portsAssociation.keySet()) {
+
+            TrafficSelector selector = DefaultTrafficSelector.builder()
+                    .matchInPhyPort(srcPoint.port())
+                    .build();
+            TrafficTreatment treatment = DefaultTrafficTreatment.builder()
+                    .setOutput(portsAssociation.get(srcPoint).port())
+                    .build();
+
+            FlowRule flowRule = DefaultFlowRule.builder()
+                    .forDevice(srcPoint.deviceId())
+                    .withSelector(selector)
+                    .withTreatment(treatment)
+                    .fromApp(APP_ID)
+                    .withPriority(DEFAULT_PRIORITY)
+                    .makePermanent()
+                    .build();
+            flowRules.add(flowRule);
+        }
+
+
+
+        List<NetworkResource> resources = ImmutableList.of(S1_S2, S2_S4);
+
+        FlowRuleIntent intent = new FlowRuleIntent(APP_ID,
+                                                   KEY1,
+                                                   flowRules,
+                                                   resources,
+                                                   PathIntent.ProtectionType.PRIMARY,
+                                                   RG1);
+
+        List<Intent> flowRuleIntents = Lists.newArrayList();
+        flowRuleIntents.add(intent);
+
+        return flowRuleIntents;
+    }
+
+    /**
+     * Generates another FlowRuleIntent, going through a different path, for testing non-disruptive reallocation.
+     *
+     * @return the FlowRuleIntents for test
+     */
+    public List<Intent> createAnotherFlowRuleIntentsNonDisruptive() {
+        Map<ConnectPoint, ConnectPoint> portsAssociation = Maps.newHashMap();
+        portsAssociation.put(CP1, CP3);
+        portsAssociation.put(CP3_1, CP3_2);
+        portsAssociation.put(CP4_3, CP4_1);
+
+        List<FlowRule> flowRules = Lists.newArrayList();
+
+        for (ConnectPoint srcPoint : portsAssociation.keySet()) {
+
+            TrafficSelector selector = DefaultTrafficSelector.builder()
+                    .matchInPhyPort(srcPoint.port())
+                    .build();
+            TrafficTreatment treatment = DefaultTrafficTreatment.builder()
+                    .setOutput(portsAssociation.get(srcPoint).port())
+                    .build();
+
+            FlowRule flowRule = DefaultFlowRule.builder()
+                    .forDevice(srcPoint.deviceId())
+                    .withSelector(selector)
+                    .withTreatment(treatment)
+                    .fromApp(APP_ID)
+                    .withPriority(DEFAULT_PRIORITY)
+                    .makePermanent()
+                    .build();
+            flowRules.add(flowRule);
+        }
+
+
+
+        List<NetworkResource> resources = ImmutableList.of(S1_S3, S3_S4);
+
+        FlowRuleIntent intent = new FlowRuleIntent(APP_ID,
+                                                   KEY1,
+                                                   flowRules,
+                                                   resources,
+                                                   PathIntent.ProtectionType.PRIMARY,
+                                                   RG1);
+
+        List<Intent> flowRuleIntents = Lists.newArrayList();
+        flowRuleIntents.add(intent);
+
+        return flowRuleIntents;
+    }
+
+    /**
      * The FlowRuleService for test; always success for any flow rule operations.
      */
     class TestFlowRuleService extends FlowRuleServiceAdapter {
@@ -489,4 +704,37 @@
         }
     }
 
+    /**
+     * The FlowRuleService for testing non-disruptive reallocation.
+     * It keeps all the FlowRules installed/uninstalled.
+     */
+    class TestFlowRuleServiceNonDisruptive extends FlowRuleServiceAdapter {
+
+        Set<FlowRule> flowRulesAdd = Sets.newHashSet();
+        Set<FlowRule> flowRulesRemove = Sets.newHashSet();
+
+        public void record(FlowRuleOperations ops) {
+            ops.stages().forEach(stage -> {
+                stage.forEach(op -> {
+                    switch (op.type()) {
+                        case ADD:
+                            flowRulesAdd.add(op.rule());
+                            break;
+                        case REMOVE:
+                            flowRulesRemove.add(op.rule());
+                            break;
+                        default:
+                            break;
+                    }
+                });
+            });
+        }
+
+        @Override
+        public void apply(FlowRuleOperations ops) {
+            record(ops);
+            ops.callback().onSuccess(ops);
+        }
+    }
+
 }
diff --git a/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java b/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java
index 793b05b..f2bf9ae 100644
--- a/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java
+++ b/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java
@@ -193,6 +193,7 @@
 import org.onosproject.net.intent.constraint.HashedPathSelectionConstraint;
 import org.onosproject.net.intent.constraint.LatencyConstraint;
 import org.onosproject.net.intent.constraint.LinkTypeConstraint;
+import org.onosproject.net.intent.constraint.NonDisruptiveConstraint;
 import org.onosproject.net.intent.constraint.ObstacleConstraint;
 import org.onosproject.net.intent.constraint.PartialFailureConstraint;
 import org.onosproject.net.intent.constraint.ProtectionConstraint;
@@ -537,6 +538,7 @@
                     EncapsulationConstraint.class,
                     EncapsulationType.class,
                     HashedPathSelectionConstraint.class,
+                    NonDisruptiveConstraint.class,
                     // Flow Objectives
                     DefaultForwardingObjective.class,
                     ForwardingObjective.Flag.class,
diff --git a/web/api/src/test/java/org/onosproject/rest/resources/IntentsResourceTest.java b/web/api/src/test/java/org/onosproject/rest/resources/IntentsResourceTest.java
index ea1fc47..28ca0a5 100644
--- a/web/api/src/test/java/org/onosproject/rest/resources/IntentsResourceTest.java
+++ b/web/api/src/test/java/org/onosproject/rest/resources/IntentsResourceTest.java
@@ -683,7 +683,7 @@
         @Override
         public boolean matchesSafely(JsonArray json) {
             boolean intentFound = false;
-            final int expectedAttributes = 5;
+            final int expectedAttributes = 6;
             for (int jsonIntentIndex = 0; jsonIntentIndex < json.size();
                  jsonIntentIndex++) {