ONOS-3023 Changing flowTable sets to map so that we can compare
stored vs. new rule when adding and removing

Change-Id: Ibd885023d550af3b2220056fbdf44ad8ec7fefda
diff --git a/core/api/src/main/java/org/onosproject/net/flow/DefaultFlowRule.java b/core/api/src/main/java/org/onosproject/net/flow/DefaultFlowRule.java
index 5492fa9..1164e05 100644
--- a/core/api/src/main/java/org/onosproject/net/flow/DefaultFlowRule.java
+++ b/core/api/src/main/java/org/onosproject/net/flow/DefaultFlowRule.java
@@ -15,6 +15,7 @@
  */
 package org.onosproject.net.flow;
 
+import com.google.common.annotations.Beta;
 import org.onosproject.core.ApplicationId;
 import org.onosproject.core.DefaultGroupId;
 import org.onosproject.core.GroupId;
@@ -284,6 +285,11 @@
         return tableId;
     }
 
+    @Beta
+    public long created() {
+        return created;
+    }
+
     public static Builder builder() {
         return new Builder();
     }
diff --git a/core/common/src/test/java/org/onosproject/store/trivial/SimpleFlowRuleStore.java b/core/common/src/test/java/org/onosproject/store/trivial/SimpleFlowRuleStore.java
index bed32a2..cb86fd5 100644
--- a/core/common/src/test/java/org/onosproject/store/trivial/SimpleFlowRuleStore.java
+++ b/core/common/src/test/java/org/onosproject/store/trivial/SimpleFlowRuleStore.java
@@ -220,6 +220,7 @@
             for (StoredFlowEntry stored : entries) {
                 if (stored.equals(rule)) {
                     synchronized (stored) {
+                        //FIXME modification of "stored" flow entry outside of flow table
                         stored.setBytes(rule.bytes());
                         stored.setLife(rule.life());
                         stored.setPackets(rule.packets());
diff --git a/core/net/src/main/java/org/onosproject/net/flow/impl/FlowRuleManager.java b/core/net/src/main/java/org/onosproject/net/flow/impl/FlowRuleManager.java
index 63ee03e..269d6b4 100644
--- a/core/net/src/main/java/org/onosproject/net/flow/impl/FlowRuleManager.java
+++ b/core/net/src/main/java/org/onosproject/net/flow/impl/FlowRuleManager.java
@@ -311,6 +311,7 @@
                     } catch (UnsupportedOperationException e) {
                         log.warn(e.getMessage());
                         if (flowRule instanceof DefaultFlowEntry) {
+                            //FIXME modification of "stored" flow entry outside of store
                             ((DefaultFlowEntry) flowRule).setState(FlowEntry.FlowEntryState.FAILED);
                         }
                     }
@@ -323,10 +324,8 @@
                 log.debug("Flow {} removed", flowRule);
                 post(event);
             }
-
         }
 
-
         private void extraneousFlow(FlowRule flowRule) {
             checkNotNull(flowRule, FLOW_RULE_NULL);
             checkValidity();
@@ -335,13 +334,11 @@
             log.debug("Flow {} is on switch but not in store.", flowRule);
         }
 
-
         private void flowAdded(FlowEntry flowEntry) {
             checkNotNull(flowEntry, FLOW_RULE_NULL);
             checkValidity();
 
             if (checkRuleLiveness(flowEntry, store.getFlowEntry(flowEntry))) {
-
                 FlowRuleEvent event = store.addOrUpdateFlowRule(flowEntry);
                 if (event == null) {
                     log.debug("No flow store event generated.");
@@ -353,7 +350,6 @@
                 log.debug("Removing flow rules....");
                 removeFlowRules(flowEntry);
             }
-
         }
 
         private boolean checkRuleLiveness(FlowEntry swRule, FlowEntry storedRule) {
diff --git a/core/net/src/test/java/org/onosproject/net/flow/impl/FlowRuleManagerTest.java b/core/net/src/test/java/org/onosproject/net/flow/impl/FlowRuleManagerTest.java
index 7ef8762..553745b 100644
--- a/core/net/src/test/java/org/onosproject/net/flow/impl/FlowRuleManagerTest.java
+++ b/core/net/src/test/java/org/onosproject/net/flow/impl/FlowRuleManagerTest.java
@@ -269,23 +269,19 @@
 
     @Test
     public void flowRemoved() {
-
         FlowRule f1 = addFlowRule(1);
         FlowRule f2 = addFlowRule(2);
         StoredFlowEntry fe1 = new DefaultFlowEntry(f1);
         FlowEntry fe2 = new DefaultFlowEntry(f2);
 
-
         providerService.pushFlowMetrics(DID, ImmutableList.of(fe1, fe2));
         service.removeFlowRules(f1);
 
+        //FIXME modification of "stored" flow entry outside of store
         fe1.setState(FlowEntryState.REMOVED);
 
-
-
         providerService.flowRemoved(fe1);
 
-
         validateEvents(RULE_ADD_REQUESTED, RULE_ADD_REQUESTED, RULE_ADDED,
                 RULE_ADDED, RULE_REMOVE_REQUESTED, RULE_REMOVED);
 
@@ -301,7 +297,6 @@
 
         providerService.flowRemoved(fe3);
         validateEvents();
-
     }
 
     @Test
diff --git a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/NewDistributedFlowRuleStore.java b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/NewDistributedFlowRuleStore.java
index 1695e5f..dc1bf86 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/NewDistributedFlowRuleStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/NewDistributedFlowRuleStore.java
@@ -15,92 +15,92 @@
  */
 package org.onosproject.store.flow.impl;
 
-import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.Futures;
+ import com.google.common.base.Objects;
+ import com.google.common.collect.ImmutableList;
+ import com.google.common.collect.ImmutableMap;
+ import com.google.common.collect.Iterables;
+ import com.google.common.collect.Maps;
+ import com.google.common.collect.Sets;
+ import com.google.common.util.concurrent.Futures;
+ 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.apache.felix.scr.annotations.Service;
+ import org.onlab.util.KryoNamespace;
+ import org.onlab.util.Tools;
+ import org.onosproject.cfg.ComponentConfigService;
+ import org.onosproject.cluster.ClusterService;
+ import org.onosproject.cluster.NodeId;
+ import org.onosproject.core.CoreService;
+ import org.onosproject.core.IdGenerator;
+ import org.onosproject.mastership.MastershipService;
+ import org.onosproject.net.DeviceId;
+ import org.onosproject.net.device.DeviceService;
+ import org.onosproject.net.flow.CompletedBatchOperation;
+ import org.onosproject.net.flow.DefaultFlowEntry;
+ import org.onosproject.net.flow.FlowEntry;
+ import org.onosproject.net.flow.FlowEntry.FlowEntryState;
+ import org.onosproject.net.flow.FlowId;
+ import org.onosproject.net.flow.FlowRule;
+ import org.onosproject.net.flow.FlowRuleBatchEntry;
+ import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
+ import org.onosproject.net.flow.FlowRuleBatchEvent;
+ import org.onosproject.net.flow.FlowRuleBatchOperation;
+ import org.onosproject.net.flow.FlowRuleBatchRequest;
+ import org.onosproject.net.flow.FlowRuleEvent;
+ import org.onosproject.net.flow.FlowRuleEvent.Type;
+ import org.onosproject.net.flow.FlowRuleService;
+ import org.onosproject.net.flow.FlowRuleStore;
+ import org.onosproject.net.flow.FlowRuleStoreDelegate;
+ import org.onosproject.net.flow.StoredFlowEntry;
+ import org.onosproject.net.flow.TableStatisticsEntry;
+ import org.onosproject.persistence.PersistenceService;
+ import org.onosproject.store.AbstractStore;
+ import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
+ import org.onosproject.store.cluster.messaging.ClusterMessage;
+ import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
+ import org.onosproject.store.flow.ReplicaInfoEvent;
+ import org.onosproject.store.flow.ReplicaInfoEventListener;
+ import org.onosproject.store.flow.ReplicaInfoService;
+ import org.onosproject.store.impl.MastershipBasedTimestamp;
+ import org.onosproject.store.serializers.KryoNamespaces;
+ import org.onosproject.store.serializers.KryoSerializer;
+ import org.onosproject.store.serializers.StoreSerializer;
+ import org.onosproject.store.serializers.custom.DistributedStoreSerializers;
+ import org.onosproject.store.service.EventuallyConsistentMap;
+ import org.onosproject.store.service.EventuallyConsistentMapEvent;
+ import org.onosproject.store.service.EventuallyConsistentMapListener;
+ import org.onosproject.store.service.Serializer;
+ import org.onosproject.store.service.StorageService;
+ import org.onosproject.store.service.WallClockTimestamp;
+ import org.osgi.service.component.ComponentContext;
+ import org.slf4j.Logger;
 
-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.apache.felix.scr.annotations.Service;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Tools;
-import org.onosproject.cfg.ComponentConfigService;
-import org.onosproject.cluster.ClusterService;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.core.CoreService;
-import org.onosproject.core.IdGenerator;
-import org.onosproject.mastership.MastershipService;
-import org.onosproject.net.DeviceId;
-import org.onosproject.net.device.DeviceService;
-import org.onosproject.net.flow.CompletedBatchOperation;
-import org.onosproject.net.flow.DefaultFlowEntry;
-import org.onosproject.net.flow.FlowEntry;
-import org.onosproject.net.flow.FlowEntry.FlowEntryState;
-import org.onosproject.net.flow.FlowId;
-import org.onosproject.net.flow.FlowRule;
-import org.onosproject.net.flow.FlowRuleBatchEntry;
-import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
-import org.onosproject.net.flow.FlowRuleBatchEvent;
-import org.onosproject.net.flow.FlowRuleBatchOperation;
-import org.onosproject.net.flow.FlowRuleBatchRequest;
-import org.onosproject.net.flow.FlowRuleEvent;
-import org.onosproject.net.flow.FlowRuleEvent.Type;
-import org.onosproject.net.flow.FlowRuleService;
-import org.onosproject.net.flow.FlowRuleStore;
-import org.onosproject.net.flow.FlowRuleStoreDelegate;
-import org.onosproject.net.flow.StoredFlowEntry;
-import org.onosproject.net.flow.TableStatisticsEntry;
-import org.onosproject.persistence.PersistenceService;
-import org.onosproject.store.AbstractStore;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.cluster.messaging.ClusterMessage;
-import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
-import org.onosproject.store.flow.ReplicaInfoEvent;
-import org.onosproject.store.flow.ReplicaInfoEventListener;
-import org.onosproject.store.flow.ReplicaInfoService;
-import org.onosproject.store.impl.MastershipBasedTimestamp;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.serializers.KryoSerializer;
-import org.onosproject.store.serializers.StoreSerializer;
-import org.onosproject.store.serializers.custom.DistributedStoreSerializers;
-import org.onosproject.store.service.EventuallyConsistentMap;
-import org.onosproject.store.service.EventuallyConsistentMapEvent;
-import org.onosproject.store.service.EventuallyConsistentMapListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.StorageService;
-import org.onosproject.store.service.WallClockTimestamp;
-import org.osgi.service.component.ComponentContext;
-import org.slf4j.Logger;
+ import java.util.Collections;
+ import java.util.Dictionary;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.ScheduledExecutorService;
+ import java.util.concurrent.ScheduledFuture;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicInteger;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.stream.Collectors;
 
-import java.util.Collections;
-import java.util.Dictionary;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Collectors;
-
-import static com.google.common.base.Strings.isNullOrEmpty;
-import static org.onlab.util.Tools.get;
-import static org.onlab.util.Tools.groupedThreads;
-import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
-import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.*;
-import static org.slf4j.LoggerFactory.getLogger;
+ import static com.google.common.base.Strings.isNullOrEmpty;
+ import static org.onlab.util.Tools.get;
+ import static org.onlab.util.Tools.groupedThreads;
+ import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
+ import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.*;
+ import static org.slf4j.LoggerFactory.getLogger;
 
 /**
  * Manages inventory of flow rules using a distributed state management protocol.
@@ -498,7 +498,9 @@
                         case REMOVE:
                             entry = flowTable.getFlowEntry(op.target());
                             if (entry != null) {
+                                //FIXME modification of "stored" flow entry outside of flow table
                                 entry.setState(FlowEntryState.PENDING_REMOVE);
+                                log.debug("Setting state of rule to pending remove: {}", entry);
                                 return op;
                             }
                             break;
@@ -539,6 +541,7 @@
         // check if this new rule is an update to an existing entry
         StoredFlowEntry stored = flowTable.getFlowEntry(rule);
         if (stored != null) {
+            //FIXME modification of "stored" flow entry outside of flow table
             stored.setBytes(rule.bytes());
             stored.setLife(rule.life());
             stored.setPackets(rule.packets());
@@ -588,8 +591,9 @@
     private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
         final DeviceId deviceId = rule.deviceId();
         // This is where one could mark a rule as removed and still keep it in the store.
-        final boolean removed = flowTable.remove(deviceId, rule); //flowEntries.remove(deviceId, rule);
-        return removed ? new FlowRuleEvent(RULE_REMOVED, rule) : null;
+        final FlowEntry removed = flowTable.remove(deviceId, rule);
+        // rule may be partial rule that is missing treatment, we should use rule from store instead
+        return removed != null ? new FlowRuleEvent(RULE_REMOVED, removed) : null;
     }
 
     @Override
@@ -635,7 +639,8 @@
 
     private class InternalFlowTable implements ReplicaInfoEventListener {
 
-        private final Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>>
+        //TODO replace the Map<V,V> with ExtendedSet
+        private final Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
                 flowEntries = Maps.newConcurrentMap();
 
         private final Map<DeviceId, Long> lastBackupTimes = Maps.newConcurrentMap();
@@ -692,30 +697,32 @@
                 return;
             }
             log.debug("Sending flowEntries for devices {} to {} as backup.", deviceIds, nodeId);
-            Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>> deviceFlowEntries =
-                    Maps.newConcurrentMap();
+            Map<DeviceId, Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>
+                    deviceFlowEntries = Maps.newConcurrentMap();
             deviceIds.forEach(id -> deviceFlowEntries.put(id, ImmutableMap.copyOf(getFlowTable(id))));
-            clusterCommunicator.<Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>>, Set<DeviceId>>sendAndReceive(
-                                        deviceFlowEntries,
-                                        FLOW_TABLE_BACKUP,
-                                        SERIALIZER::encode,
-                                        SERIALIZER::decode,
-                                        nodeId)
-                               .whenComplete((backedupDevices, error) -> {
-                                   Set<DeviceId> devicesNotBackedup = error != null ?
-                                           deviceFlowEntries.keySet() :
-                                           Sets.difference(deviceFlowEntries.keySet(), backedupDevices);
-                                   if (devicesNotBackedup.size() > 0) {
-                                       log.warn("Failed to backup devices: {}. Reason: {}",
-                                               devicesNotBackedup, error.getMessage());
-                                   }
-                                   if (backedupDevices != null) {
-                                       backedupDevices.forEach(id -> {
-                                           lastBackupTimes.put(id, System.currentTimeMillis());
-                                           lastBackupNodes.put(id, nodeId);
-                                       });
-                                   }
-                               });
+            clusterCommunicator.<Map<DeviceId,
+                                 Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>,
+                                 Set<DeviceId>>
+                    sendAndReceive(deviceFlowEntries,
+                                   FLOW_TABLE_BACKUP,
+                                   SERIALIZER::encode,
+                                   SERIALIZER::decode,
+                                   nodeId)
+                    .whenComplete((backedupDevices, error) -> {
+                        Set<DeviceId> devicesNotBackedup = error != null ?
+                            deviceFlowEntries.keySet() :
+                            Sets.difference(deviceFlowEntries.keySet(), backedupDevices);
+                        if (devicesNotBackedup.size() > 0) {
+                            log.warn("Failed to backup devices: {}. Reason: {}",
+                                     devicesNotBackedup, error.getMessage());
+                        }
+                        if (backedupDevices != null) {
+                            backedupDevices.forEach(id -> {
+                                lastBackupTimes.put(id, System.currentTimeMillis());
+                                lastBackupNodes.put(id, nodeId);
+                            });
+                        }
+                    });
         }
 
         /**
@@ -724,10 +731,10 @@
          * @param deviceId identifier of the device
          * @return Map representing Flow Table of given device.
          */
-        private Map<FlowId, Set<StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
+        private Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
             if (persistenceEnabled) {
                 return flowEntries.computeIfAbsent(deviceId, id -> persistenceService
-                        .<FlowId, Set<StoredFlowEntry>>persistentMapBuilder()
+                        .<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>persistentMapBuilder()
                         .withName("FlowTable:" + deviceId.toString())
                         .withSerializer(new Serializer() {
                             @Override
@@ -746,22 +753,18 @@
             }
         }
 
-        private Set<StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
-            return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Sets.newCopyOnWriteArraySet());
+        private Map<StoredFlowEntry, StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
+            return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Maps.newConcurrentMap());
         }
 
         private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
-            Set<StoredFlowEntry> flowEntries = getFlowEntriesInternal(rule.deviceId(), rule.id());
-            return flowEntries.stream()
-                              .filter(entry -> Objects.equal(entry, rule))
-                              .findAny()
-                              .orElse(null);
+            return getFlowEntriesInternal(rule.deviceId(), rule.id()).get(rule);
         }
 
         private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
-            Set<FlowEntry> result = Sets.newHashSet();
-            getFlowTable(deviceId).values().forEach(result::addAll);
-            return result;
+            return getFlowTable(deviceId).values().stream()
+                        .flatMap(m -> m.values().stream())
+                        .collect(Collectors.toSet());
         }
 
         public StoredFlowEntry getFlowEntry(FlowRule rule) {
@@ -773,15 +776,40 @@
         }
 
         public void add(FlowEntry rule) {
-            getFlowEntriesInternal(rule.deviceId(), rule.id()).add((StoredFlowEntry) rule);
+            getFlowEntriesInternal(rule.deviceId(), rule.id())
+                    .compute((StoredFlowEntry) rule, (k, stored) -> {
+                        //TODO compare stored and rule timestamps
+                        //TODO the key is not updated
+                        return (StoredFlowEntry) rule;
+                    });
             lastUpdateTimes.put(rule.deviceId(), System.currentTimeMillis());
         }
 
-        public boolean remove(DeviceId deviceId, FlowEntry rule) {
-            try {
-                return getFlowEntriesInternal(deviceId, rule.id()).remove(rule);
-            } finally {
+        public FlowEntry remove(DeviceId deviceId, FlowEntry rule) {
+            final AtomicReference<FlowEntry> removedRule = new AtomicReference<>();
+            getFlowEntriesInternal(rule.deviceId(), rule.id())
+                .computeIfPresent((StoredFlowEntry) rule, (k, stored) -> {
+                    if (rule instanceof DefaultFlowEntry) {
+                        DefaultFlowEntry toRemove = (DefaultFlowEntry) rule;
+                        if (stored instanceof DefaultFlowEntry) {
+                            DefaultFlowEntry storedEntry = (DefaultFlowEntry) stored;
+                            if (toRemove.created() < storedEntry.created()) {
+                                log.debug("Trying to remove more recent flow entry {} (stored: {})",
+                                          toRemove, stored);
+                                // the key is not updated, removedRule remains null
+                                return stored;
+                            }
+                        }
+                    }
+                    removedRule.set(stored);
+                    return null;
+                });
+
+            if (removedRule.get() != null) {
                 lastUpdateTimes.put(deviceId, System.currentTimeMillis());
+                return removedRule.get();
+            } else {
+                return null;
             }
         }
 
@@ -826,14 +854,16 @@
             }
         }
 
-        private Set<DeviceId> onBackupReceipt(Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>> flowTables) {
+        private Set<DeviceId> onBackupReceipt(Map<DeviceId,
+                Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>> flowTables) {
             log.debug("Received flowEntries for {} to backup", flowTables.keySet());
             Set<DeviceId> backedupDevices = Sets.newHashSet();
             try {
                 flowTables.forEach((deviceId, deviceFlowTable) -> {
                     // Only process those devices are that not managed by the local node.
                     if (!Objects.equal(local, mastershipService.getMasterFor(deviceId))) {
-                        Map<FlowId, Set<StoredFlowEntry>> backupFlowTable = getFlowTable(deviceId);
+                        Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>> backupFlowTable =
+                                getFlowTable(deviceId);
                         backupFlowTable.clear();
                         backupFlowTable.putAll(deviceFlowTable);
                         backedupDevices.add(deviceId);
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/NewAdaptiveFlowStatsCollector.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/NewAdaptiveFlowStatsCollector.java
index aa8df94..e286684 100644
--- a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/NewAdaptiveFlowStatsCollector.java
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/NewAdaptiveFlowStatsCollector.java
@@ -416,6 +416,7 @@
                                    + " AdaptiveStats collection thread for {}",
                            sw.getStringId());
 
+                   //FIXME modification of "stored" flow entry outside of store
                    stored.setLastSeen();
                    continue;
                } else if (fe.life() < stored.life()) {
@@ -428,11 +429,13 @@
                                ", new life=" + fe.life() + ", old life=" + stored.life() +
                                ", new lastSeen=" + fe.lastSeen() + ", old lastSeen=" + stored.lastSeen());
                    // go next
+                   //FIXME modification of "stored" flow entry outside of store
                    stored.setLastSeen();
                    continue;
                }
 
                // update now
+               //FIXME modification of "stored" flow entry outside of store
                stored.setLife(fe.life());
                stored.setPackets(fe.packets());
                stored.setBytes(fe.bytes());