[newOpenTAM] new Upgraded AFM and FlowStatisticService

New Upgraded AFM(Adaptive Flow Monitoring) and FlowStatisticService.
  .Reabsed from master 2016.12.06, and assumed avgPollInterval in FlowStatisticService with flowPollFrequency in case adativeFlowSampling is true or false
  .Fixed Yuta HIGUCHI comments, 2016.12.03
  .Fixed checkstyle warning
  .Rebased and added interrupted flag for exiting while() loop when IterruptException is caught in NewAdaptiveFlowStatsCollector.java

  .Fixed and added javadocs from Thomas Vachuska's comment
  .Removed synchronized block in OpenFlowRuleProvider.java for avoiding performance degradation
  .Rebased from master 2016.10.13 1.8.0.SNAPSHOT
  .Rebased from master 2016.09.09
  .Fixed Yuta HIGUCHI's comments
   (made default getFlowEntriesByLiveType interface,
    added checkArgument() for StoredFlowEntry,
    added @Deprecated annotation @deprecated javadoc,
    added Thread.currentThread.interrupt()
    and fixed Default Adaptive Flow Sampling value with false.)

  .Rebased from master branch
  .Fix typo in FlowRuleService.java comment line
  .Quick Bug fix in NewAdaptiveFlowStatCollector
  .master rebased for fixing build Failure
  .Removed synchronized block in FlowRuleProvider and NewAdaptiveFlowStatCollector for performance upgrade

  .Removed duplicated flow entries in NewAdaptiveFlowStatsCollector
  .Removed additional operation (add/remove/mod) of flow entry in OpenFlowRuleProvider
  .Set default adaptiveFlowSampling value with true
  .Added liveType (IMMEDIATE, SHORT, MID, LONG) member variable in FlowEntry
  .New added PollInterval static class for pollInterval value adjustment
  .Updated FlowEntryBuilder and FlowEntry constructor
  .Added liveType print in FlowListCommand CLI
  .Removed FlowStatisticStore, used existing StatisticStore
  .New added FlowEntryWithLoad for replacing the old TypedFlowEntryWithLoad
  .Added new interfaces in FlowStatisticService
  .Updated GetFlowStatistics CLI for using new interfaces
  .All Typedxxx classes are deprecated

  .new created review 9292 from review 9232
  .fixed Jian Li's review comment from review 9232
  .fixed Build failure in core/net/BUCK file

Change-Id: I7a0e39c5220a2b279b68a195347c183b5bdf1a49
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 52d6099..8d885a1 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
@@ -16,19 +16,18 @@
 
 package org.onosproject.provider.of.flow.impl;
 
-import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+import com.google.common.collect.Iterables;
+import org.onlab.osgi.DefaultServiceDirectory;
+import org.onosproject.net.DeviceId;
 import org.onosproject.net.driver.DriverService;
-import org.onosproject.net.flow.DefaultTypedFlowEntry;
 import org.onosproject.net.flow.FlowEntry;
-import org.onosproject.net.flow.FlowId;
-import org.onosproject.net.flow.FlowRule;
+import org.onosproject.net.flow.FlowRuleService;
 import org.onosproject.net.flow.StoredFlowEntry;
-import org.onosproject.net.flow.TypedStoredFlowEntry;
 import org.onosproject.net.flow.instructions.Instruction;
 import org.onosproject.net.flow.instructions.Instructions;
+import org.onosproject.net.statistic.DefaultLoad;
+import org.onosproject.net.statistic.PollInterval;
+import org.onosproject.openflow.controller.Dpid;
 import org.onosproject.openflow.controller.OpenFlowSwitch;
 import org.onosproject.openflow.controller.RoleState;
 import org.projectfloodlight.openflow.protocol.OFFlowStatsRequest;
@@ -37,19 +36,15 @@
 import org.projectfloodlight.openflow.types.TableId;
 import org.slf4j.Logger;
 
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static java.lang.Thread.sleep;
 import static org.onlab.util.Tools.groupedThreads;
-import static org.onosproject.net.flow.TypedStoredFlowEntry.FlowLiveType;
 import static org.slf4j.LoggerFactory.getLogger;
 
 /**
@@ -64,18 +59,16 @@
                     + ", priority={}, selector={}, treatment={} dpid={}";
 
     private static final String CHECK_AND_MOVE_COUNT_LOG =
-            "checkAndMoveLiveFlowAll: Total Flow_Count={}, add-remove_Count={}"
+            "checkAndMoveLiveFlowAll: Total Flow_Count={}, "
                     + ", IMMEDIATE_FLOW_Count={}, SHORT_FLOW_Count={}"
-                    + ", MID_FLOW_Count={}, LONG_FLOW_Count={}, add_Count={}"
-                    + ", addWithSetFlowLiveType_Count={}, remove_Count={}, dpid={}";
+                    + ", MID_FLOW_Count={}, LONG_FLOW_Count={}, UNKNOWN_FLOW_Count={}";
 
-    private static final String ADD_INVALID_LOG =
-            "addOrUpdateFlows: invalid flow update! The new life is SMALLER than the previous one"
-                    + ", new flowId={}, old flowId={}, new bytes={}, old bytes={}"
-                    + ", new life={}, old life={}, new lastSeen={}, old lastSeen={}";
+    private static final int SLEEP_LOOP_COUNT = 10;
+    private static final int SLEEP_MS = 100;
 
     private final DriverService driverService;
     private final OpenFlowSwitch sw;
+    private final DeviceId did;
 
     private ScheduledExecutorService adaptiveFlowStatsScheduler =
             Executors.newScheduledThreadPool(4, groupedThreads("onos/flow", "device-stats-collector-%d", log));
@@ -113,13 +106,13 @@
     private int callCountMidFlowsTask = 0;   // increased MID_POLL_TIMES whenever Task is called
     private int callCountLongFlowsTask = 0;  // increased LONG_POLL_TIMES whenever Task is called
 
-    private InternalDeviceFlowTable deviceFlowTable = new InternalDeviceFlowTable();
-
     private boolean isFirstTimeStart = true;
 
     public static final long NO_FLOW_MISSING_XID = (-1);
     private long flowMissingXid = NO_FLOW_MISSING_XID;
 
+    private FlowRuleService flowRuleService;
+
     /**
      * Creates a new adaptive collector for the given switch and default cal_and_poll frequency.
      *
@@ -130,9 +123,25 @@
     NewAdaptiveFlowStatsCollector(DriverService driverService, OpenFlowSwitch sw, int pollInterval) {
         this.driverService = driverService;
         this.sw = sw;
+        this.did = DeviceId.deviceId(Dpid.uri(sw.getId()));
+
+        flowRuleService = get(FlowRuleService.class);
+
         initMemberVars(pollInterval);
     }
 
+    /**
+     * Returns the reference to the implementation of the specified service.
+     *
+     * @param serviceClass service class
+     * @param <T>          type of service
+     * @return service implementation
+     * @throws org.onlab.osgi.ServiceNotFoundException if service is unavailable
+     */
+    private static <T> T get(Class<T> serviceClass) {
+        return DefaultServiceDirectory.getService(serviceClass);
+    }
+
     // check calAndPollInterval validity and set all pollInterval values and finally initialize each task call count
     private void initMemberVars(int pollInterval) {
         if (pollInterval < MIN_CAL_AND_POLL_FREQUENCY) {
@@ -148,6 +157,16 @@
         longPollInterval = LONG_POLL_TIMES * calAndPollInterval;
         entirePollInterval = ENTIRE_POLL_TIMES * calAndPollInterval;
 
+        // Set the PollInterval values for statistic manager and others usage
+        DefaultLoad.setPollInterval(calAndPollInterval);
+
+        PollInterval pollInterval1Instance = PollInterval.getInstance();
+
+        pollInterval1Instance.setPollInterval(calAndPollInterval);
+        pollInterval1Instance.setMidPollInterval(midPollInterval);
+        pollInterval1Instance.setLongPollInterval(longPollInterval);
+        pollInterval1Instance.setEntirePollInterval(entirePollInterval);
+
         callCountCalAndShortFlowsTask = 0;
         callCountMidFlowsTask = 0;
         callCountLongFlowsTask = 0;
@@ -228,23 +247,21 @@
     }
 
     // send openflow flow stats request message with getting all flow entries to a given switch sw
-    private void ofFlowStatsRequestAllSend() {
+    private synchronized void ofFlowStatsRequestAllSend() {
         OFFlowStatsRequest request = sw.factory().buildFlowStatsRequest()
                 .setMatch(sw.factory().matchWildcardAll())
                 .setTableId(TableId.ALL)
                 .setOutPort(OFPort.NO_MASK)
                 .build();
 
-        synchronized (this) {
-            // set the request xid to check the reply in OpenFlowRuleProvider
-            // After processing the reply of this request message,
-            // this must be set to NO_FLOW_MISSING_XID(-1) by provider
-            setFlowMissingXid(request.getXid());
-            log.debug("ofFlowStatsRequestAllSend: request={}, dpid={}",
+        // set the request xid to check the reply in OpenFlowRuleProvider
+        // After processing the reply of this request message,
+        // this must be set to NO_FLOW_MISSING_XID(-1) by provider
+        setFlowMissingXid(request.getXid());
+        log.debug("ofFlowStatsRequestAllSend: request={}, dpid={}",
                     request.toString(), sw.getStringId());
 
-            sw.sendMsg(request);
-        }
+        sw.sendMsg(request);
     }
 
     // send openflow flow stats request message with getting the specific flow entry(fe) to a given switch sw
@@ -271,22 +288,54 @@
                 .setOutPort(ofPort)
                 .build();
 
-        synchronized (this) {
-            if (getFlowMissingXid() != NO_FLOW_MISSING_XID) {
-                log.debug("ofFlowStatsRequestFlowSend: previous FlowStatsRequestAll does not be processed yet,"
-                                + " set no flow missing xid anyway, for {}",
-                        sw.getStringId());
-                setFlowMissingXid(NO_FLOW_MISSING_XID);
-            }
+        // Wait for 1 second until the FlowRuleProvider finishes to process FlowStatReply message
+        int loop = 0;
+        boolean interrupted = false;
+        while (!interrupted && getFlowMissingXid() != NO_FLOW_MISSING_XID) {
+            if (loop++ < SLEEP_LOOP_COUNT) {
+                log.debug("ofFlowStatsRequestFlowSend: previous FlowStatsRequestAll (xid={})" +
+                                  " does not be processed yet, do sleep for {} ms, for {}",
+                          getFlowMissingXid(),
+                          SLEEP_MS,
+                          sw.getStringId());
+                try {
+                    sleep(SLEEP_MS);
+                } catch (InterruptedException ie) {
+                    log.debug("ofFlowStatsRequestFlowSend: Interrupted Exception = {}, for {}",
+                              ie.toString(),
+                              sw.getStringId());
+                    // for exiting while loop gracefully
+                    interrupted = true;
+                }
+            } else {
+                log.debug("ofFlowStatsRequestFlowSend: previous FlowStatsRequestAll (xid={})" +
+                                  " does not be processed yet, for {} ms," +
+                                  " just set xid with NO_FLOW_MISSING_XID, for {}",
+                          getFlowMissingXid(),
+                          loop * SLEEP_MS,
+                          sw.getStringId());
 
-            sw.sendMsg(request);
+                setFlowMissingXid(NO_FLOW_MISSING_XID);
+                break;
+            }
         }
+
+        sw.sendMsg(request);
+
     }
 
     private void calAndShortFlowsTaskInternal() {
-        deviceFlowTable.checkAndMoveLiveFlowAll();
+        checkAndMoveLiveFlowAll();
 
-        deviceFlowTable.getShortFlows().forEach(fe -> {
+        ofFlowStatsRequestInternal(FlowEntry.FlowLiveType.SHORT);
+    }
+
+    private void ofFlowStatsRequestInternal(FlowEntry.FlowLiveType liveType) {
+
+        Iterable<FlowEntry> flowEntries =
+                flowRuleService.getFlowEntriesByLiveType(did, liveType);
+
+        flowEntries.forEach(fe -> {
             ofFlowStatsRequestFlowSend(fe);
         });
     }
@@ -309,9 +358,7 @@
     }
 
     private void midFlowsTaskInternal() {
-        deviceFlowTable.getMidFlows().forEach(fe -> {
-            ofFlowStatsRequestFlowSend(fe);
-        });
+        ofFlowStatsRequestInternal(FlowEntry.FlowLiveType.MID);
     }
 
     private class LongFlowsTask implements Runnable {
@@ -332,9 +379,7 @@
     }
 
     private void longFlowsTaskInternal() {
-        deviceFlowTable.getLongFlows().forEach(fe -> {
-            ofFlowStatsRequestFlowSend(fe);
-        });
+        ofFlowStatsRequestInternal(FlowEntry.FlowLiveType.LONG);
     }
 
     /**
@@ -396,111 +441,6 @@
     }
 
     /**
-     * Adds typed flow entry from flow rule into the internal flow table.
-     *
-     * @param flowRules the flow rules
-     */
-    public synchronized void addWithFlowRule(FlowRule... flowRules) {
-        for (FlowRule fr : flowRules) {
-            // First remove old entry unconditionally, if exist
-            deviceFlowTable.remove(fr);
-
-            // add new flow entry, we suppose IMMEDIATE_FLOW
-            TypedStoredFlowEntry newFlowEntry = new DefaultTypedFlowEntry(fr,
-                    FlowLiveType.IMMEDIATE_FLOW);
-            deviceFlowTable.addWithCalAndSetFlowLiveType(newFlowEntry);
-        }
-    }
-
-    /**
-     * Adds or updates typed flow entry from flow entry into the internal flow table.
-     *
-     * @param flowEntries the flow entries
-     */
-    public synchronized void addOrUpdateFlows(FlowEntry... flowEntries) {
-        for (FlowEntry fe : flowEntries) {
-            // check if this new rule is an update to an existing entry
-            TypedStoredFlowEntry stored = deviceFlowTable.getFlowEntry(fe);
-
-            if (stored != null) {
-                // duplicated flow entry is collected!, just skip
-                if (fe.bytes() == stored.bytes() && fe.packets() == stored.packets()
-                        && fe.life() == stored.life()) {
-                    if (log.isTraceEnabled()) {
-                        log.trace("addOrUpdateFlows({}): flowId={},is DUPLICATED stats collection, just skip.",
-                                sw.getStringId(), fe.id());
-                    }
-
-                    //FIXME modification of "stored" flow entry outside of store
-                    stored.setLastSeen();
-                    continue;
-                } else if (fe.life() < stored.life()) {
-                    // Invalid updates the stats values, i.e., bytes, packets, durations ...
-                    if (log.isDebugEnabled()) {
-                        log.debug(ADD_INVALID_LOG, fe.id(), stored.id(), fe.bytes(),
-                                stored.bytes(), fe.life(), stored.life(),
-                                fe.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(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
-                stored.setPackets(fe.packets());
-                stored.setBytes(fe.bytes());
-                stored.setLastSeen();
-                if (stored.state() == FlowEntry.FlowEntryState.PENDING_ADD) {
-                    // flow is really RULE_ADDED
-                    stored.setState(FlowEntry.FlowEntryState.ADDED);
-                }
-                // flow is RULE_UPDATED, skip adding and just updating flow live table
-                //deviceFlowTable.calAndSetFlowLiveType(stored);
-                continue;
-            }
-
-            // add new flow entry, we suppose IMMEDIATE_FLOW
-            TypedStoredFlowEntry newFlowEntry = new DefaultTypedFlowEntry(fe,
-                    FlowLiveType.IMMEDIATE_FLOW);
-            deviceFlowTable.addWithCalAndSetFlowLiveType(newFlowEntry);
-        }
-    }
-
-    /**
-     * Removes typed flow entry from the internal flow table.
-     *
-     * @param flowRules the flow entries
-     */
-    public synchronized void removeFlows(FlowRule...  flowRules) {
-        for (FlowRule rule : flowRules) {
-            deviceFlowTable.remove(rule);
-        }
-    }
-
-    // same as removeFlows() function
-    /**
-     * Removes typed flow entry from the internal flow table.
-     *
-     * @param flowRules the flow entries
-     */
-    public void flowRemoved(FlowRule... flowRules) {
-        removeFlows(flowRules);
-    }
-
-    // same as addOrUpdateFlows() function
-    /**
-     * Adds or updates typed flow entry from flow entry into the internal flow table.
-     *
-     * @param flowEntries the flow entry list
-     */
-    public void pushFlowMetrics(List<FlowEntry> flowEntries) {
-        flowEntries.forEach(this::addOrUpdateFlows);
-    }
-
-    /**
      * Returns flowMissingXid that indicates the execution of flowMissing process or not(NO_FLOW_MISSING_XID(-1)).
      *
      * @return xid of missing flow
@@ -518,339 +458,146 @@
         this.flowMissingXid = flowMissingXid;
     }
 
-    private class InternalDeviceFlowTable {
+    /**
+     * Calculates the flow live type.
+     *
+     * @param life the flow life time in seconds
+     * @return computed flow live type
+     */
+    public FlowEntry.FlowLiveType calFlowLiveType(long life) {
+        if (life < 0) {
+            return FlowEntry.FlowLiveType.UNKNOWN;
+        } else if (life < calAndPollInterval) {
+            return FlowEntry.FlowLiveType.IMMEDIATE;
+        } else if (life < midPollInterval) {
+            return FlowEntry.FlowLiveType.SHORT;
+        } else if (life < longPollInterval) {
+            return FlowEntry.FlowLiveType.MID;
+        } else { // >= longPollInterval
+            return FlowEntry.FlowLiveType.LONG;
+        }
+    }
 
-        private final Map<FlowId, Set<TypedStoredFlowEntry>>
-                flowEntries = Maps.newConcurrentMap();
+    /**
+     * Calculates and set the flow live type.
+     * It maybe called pushFlowMetrics of FlowRuleService for the ReplyFlowStat message
+     * at the first time and every entire polling time.
+     *
+     * @param fe the flow entry rule
+     * @return computed flow live type
+     */
+    public FlowEntry.FlowLiveType calAndSetFlowLiveType(StoredFlowEntry fe) {
+        checkNotNull(fe);
 
-        private final Set<StoredFlowEntry> shortFlows = new HashSet<>();
-        private final Set<StoredFlowEntry> midFlows = new HashSet<>();
-        private final Set<StoredFlowEntry> longFlows = new HashSet<>();
+        long life = fe.life();
 
-        // Assumed latency adjustment(default=500 millisecond) between FlowStatsRequest and Reply
-        private final long latencyFlowStatsRequestAndReplyMillis = 500;
-
-
-        // Statistics for table operation
-        private long addCount = 0, addWithSetFlowLiveTypeCount = 0;
-        private long removeCount = 0;
-
-        /**
-         * Resets all count values to zero.
-         */
-        public void resetAllCount() {
-            addCount = 0;
-            addWithSetFlowLiveTypeCount = 0;
-            removeCount = 0;
+        if (life < 0) {
+            fe.setLiveType(FlowEntry.FlowLiveType.UNKNOWN);
+        } else if (life < calAndPollInterval) {
+            fe.setLiveType(FlowEntry.FlowLiveType.IMMEDIATE);
+        } else if (life < midPollInterval) {
+            fe.setLiveType(FlowEntry.FlowLiveType.SHORT);
+        } else if (life < longPollInterval) {
+            fe.setLiveType(FlowEntry.FlowLiveType.MID);
+        } else { // >= longPollInterval
+            fe.setLiveType(FlowEntry.FlowLiveType.LONG);
         }
 
-        // get set of flow entries for the given flowId
-        private Set<TypedStoredFlowEntry> getFlowEntriesInternal(FlowId flowId) {
-            return flowEntries.computeIfAbsent(flowId, id -> Sets.newCopyOnWriteArraySet());
-        }
+        return fe.liveType();
+    }
 
-        // get flow entry for the given flow rule
-        private TypedStoredFlowEntry getFlowEntryInternal(FlowRule rule) {
-            Set<TypedStoredFlowEntry> flowEntries = getFlowEntriesInternal(rule.id());
-            return flowEntries.stream()
-                    .filter(entry -> Objects.equal(entry, rule))
-                    .findAny()
-                    .orElse(null);
-        }
+    /**
+     * Check and move live type for all type flow entries in table at every calAndPollInterval time.
+     *
+     */
+    private void checkAndMoveLiveFlowAll() {
 
-        // get the flow entries for all flows in flow table
-        private Set<TypedStoredFlowEntry> getFlowEntriesInternal() {
-            Set<TypedStoredFlowEntry> result = Sets.newHashSet();
+        Iterable<FlowEntry> flowEntries = flowRuleService.getFlowEntries(did);
 
-            flowEntries.values().forEach(result::addAll);
-            return result;
-        }
+        flowEntries.forEach(fe -> {
+            checkAndMoveLiveFlowInternal((StoredFlowEntry) fe);
+        });
 
-        /**
-         * Gets the number of flow entry in flow table.
-         *
-         * @return the number of flow entry
-         */
-        public long getFlowCount() {
-            return flowEntries.values().stream().mapToLong(Set::size).sum();
-        }
+        // print table counts for debug
+        if (log.isDebugEnabled()) {
+            Iterable<FlowEntry> fes;
+            synchronized (this) {
+                long totalFlowCount = flowRuleService.getFlowRuleCount();
+                fes = flowRuleService.getFlowEntriesByLiveType(
+                        did, FlowEntry.FlowLiveType.IMMEDIATE);
+                long immediateFlowCount = Iterables.size(fes);
+                fes = flowRuleService.getFlowEntriesByLiveType(
+                        did, FlowEntry.FlowLiveType.SHORT);
+                long shortFlowCount = Iterables.size(fes);
+                fes = flowRuleService.getFlowEntriesByLiveType(
+                        did, FlowEntry.FlowLiveType.MID);
+                long midFlowCount = Iterables.size(fes);
+                fes = flowRuleService.getFlowEntriesByLiveType(
+                        did, FlowEntry.FlowLiveType.LONG);
+                long longFlowCount = Iterables.size(fes);
+                fes = flowRuleService.getFlowEntriesByLiveType(
+                        did, FlowEntry.FlowLiveType.UNKNOWN);
+                long unknownFlowCount = Iterables.size(fes);
 
-        /**
-         * Gets the number of flow entry in flow table.
-         *
-         * @param rule the flow rule
-         * @return the typed flow entry
-         */
-        public TypedStoredFlowEntry getFlowEntry(FlowRule rule) {
-            checkNotNull(rule);
+                log.trace(CHECK_AND_MOVE_COUNT_LOG, totalFlowCount,
+                            immediateFlowCount, shortFlowCount, midFlowCount, longFlowCount, unknownFlowCount);
 
-            return getFlowEntryInternal(rule);
-        }
-
-        /**
-         * Gets the all typed flow entries in flow table.
-         *
-         * @return the set of typed flow entry
-         */
-        public Set<TypedStoredFlowEntry> getFlowEntries() {
-            return getFlowEntriesInternal();
-        }
-
-        /**
-         * Gets the short typed flow entries in flow table.
-         *
-         * @return the set of typed flow entry
-         */
-        public Set<StoredFlowEntry> getShortFlows() {
-            return ImmutableSet.copyOf(shortFlows);
-        }
-
-        /**
-         * Gets the mid typed flow entries in flow table.
-         *
-         * @return the set of typed flow entry
-         */
-        public Set<StoredFlowEntry> getMidFlows() {
-            return ImmutableSet.copyOf(midFlows);
-        }
-
-        /**
-         * Gets the long typed flow entries in flow table.
-         *
-         * @return the set of typed flow entry
-         */
-        public Set<StoredFlowEntry> getLongFlows() {
-            return ImmutableSet.copyOf(longFlows);
-        }
-
-        /**
-         * Add typed flow entry into table only.
-         *
-         * @param rule the flow rule
-         */
-        public synchronized void add(TypedStoredFlowEntry rule) {
-            checkNotNull(rule);
-
-            //rule have to be new DefaultTypedFlowEntry
-            boolean result = getFlowEntriesInternal(rule.id()).add(rule);
-
-            if (result) {
-                addCount++;
-            }
-        }
-
-        /**
-         * Calculates and sets the flow live type at the first time,
-         * and then add it into a corresponding typed flow table.
-         *
-         * @param rule the flow rule
-         */
-        public void calAndSetFlowLiveType(TypedStoredFlowEntry rule) {
-            checkNotNull(rule);
-
-            calAndSetFlowLiveTypeInternal(rule);
-        }
-
-        /**
-         * Adds the typed flow entry into table, and calculates and set the flow live type,
-         * and then add it into a corresponding typed flow table.
-         *
-         * @param rule the flow rule
-         */
-        public synchronized void addWithCalAndSetFlowLiveType(TypedStoredFlowEntry rule) {
-            checkNotNull(rule);
-
-            //rule have to be new DefaultTypedFlowEntry
-            boolean result = getFlowEntriesInternal(rule.id()).add(rule);
-            if (result) {
-                calAndSetFlowLiveTypeInternal(rule);
-                addWithSetFlowLiveTypeCount++;
-            } else {
-                if (log.isDebugEnabled()) {
-                    log.debug("FlowId {} ADD failed, it may already exist in table - {}",
-                            rule.id(), sw.getStringId());
+                if (immediateFlowCount < 0) {
+                    log.error("Immediate flow count is negative");
                 }
             }
         }
+        log.trace("checkAndMoveLiveFlowAll, AdaptiveStats for {}", sw.getStringId());
+    }
 
-        // In real, calculates and set the flow live type at the first time,
-        // and then add it into a corresponding typed flow table
-        private void calAndSetFlowLiveTypeInternal(TypedStoredFlowEntry rule) {
-            long life = rule.life();
-            FlowLiveType prevFlowLiveType = rule.flowLiveType();
+    // check and set the flow live type based on current time
+    private boolean checkAndMoveLiveFlowInternal(StoredFlowEntry fe) {
+        long fromLastSeen = ((System.currentTimeMillis() - fe.lastSeen()) / 1000);
+        // fe.life() unit is SECOND!
+        long liveTime = fe.life() + fromLastSeen;
 
-            if (life >= longPollInterval) {
-                rule.setFlowLiveType(FlowLiveType.LONG_FLOW);
-                longFlows.add(rule);
-            } else if (life >= midPollInterval) {
-                rule.setFlowLiveType(FlowLiveType.MID_FLOW);
-                midFlows.add(rule);
-            } else if (life >= calAndPollInterval) {
-                rule.setFlowLiveType(FlowLiveType.SHORT_FLOW);
-                shortFlows.add(rule);
-            } else if (life >= 0) {
-                rule.setFlowLiveType(FlowLiveType.IMMEDIATE_FLOW);
-            } else { // life < 0
-                rule.setFlowLiveType(FlowLiveType.UNKNOWN_FLOW);
-            }
+        FlowEntry.FlowLiveType oldLiveType = fe.liveType();
 
-            if (rule.flowLiveType() != prevFlowLiveType) {
-                switch (prevFlowLiveType) {
-                // delete it from previous flow table
-                case SHORT_FLOW:
-                    shortFlows.remove(rule);
-                    break;
-                case MID_FLOW:
-                    midFlows.remove(rule);
-                    break;
-                case LONG_FLOW:
-                    longFlows.remove(rule);
-                    break;
-                default:
-                    break;
-                }
-            }
-        }
-
-
-        // check the flow live type based on current time, then set and add it into corresponding table
-        private boolean checkAndMoveLiveFlowInternal(TypedStoredFlowEntry fe, long cTime) {
-            long curTime = (cTime > 0 ? cTime : System.currentTimeMillis());
-            // For latency adjustment(default=500 millisecond) between FlowStatsRequest and Reply
-            long fromLastSeen = ((curTime - fe.lastSeen() + latencyFlowStatsRequestAndReplyMillis) / 1000);
-            // fe.life() unit is SECOND!
-            long liveTime = fe.life() + fromLastSeen;
-
-
-            switch (fe.flowLiveType()) {
-            case IMMEDIATE_FLOW:
-                if (liveTime >= longPollInterval) {
-                    fe.setFlowLiveType(FlowLiveType.LONG_FLOW);
-                    longFlows.add(fe);
-                } else if (liveTime >= midPollInterval) {
-                    fe.setFlowLiveType(FlowLiveType.MID_FLOW);
-                    midFlows.add(fe);
-                } else if (liveTime >= calAndPollInterval) {
-                    fe.setFlowLiveType(FlowLiveType.SHORT_FLOW);
-                    shortFlows.add(fe);
+        switch (fe.liveType()) {
+            case IMMEDIATE:
+                if (liveTime >= calAndPollInterval) {
+                    fe.setLiveType(FlowEntry.FlowLiveType.SHORT);
                 }
                 break;
-            case SHORT_FLOW:
-                if (liveTime >= longPollInterval) {
-                    fe.setFlowLiveType(FlowLiveType.LONG_FLOW);
-                    shortFlows.remove(fe);
-                    longFlows.add(fe);
-                } else if (liveTime >= midPollInterval) {
-                    fe.setFlowLiveType(FlowLiveType.MID_FLOW);
-                    shortFlows.remove(fe);
-                    midFlows.add(fe);
+            case SHORT:
+                if (liveTime >= midPollInterval) {
+                    fe.setLiveType(FlowEntry.FlowLiveType.MID);
                 }
                 break;
-            case MID_FLOW:
+            case MID:
                 if (liveTime >= longPollInterval) {
-                    fe.setFlowLiveType(FlowLiveType.LONG_FLOW);
-                    midFlows.remove(fe);
-                    longFlows.add(fe);
+                    fe.setLiveType(FlowEntry.FlowLiveType.LONG);
                 }
                 break;
-            case LONG_FLOW:
+            case LONG:
                 if (fromLastSeen > entirePollInterval) {
-                    log.trace("checkAndMoveLiveFlowInternal: flow is already removed at switch.");
+                    log.trace("checkAndMoveLiveFlowInternal, flow may be already removed at switch.");
                     return false;
                 }
                 break;
-            case UNKNOWN_FLOW: // Unknown flow is an internal error flow type, just fall through
+            case UNKNOWN: // Unknown live type is calculated and set with correct flow live type here.
+                calAndSetFlowLiveType(fe);
+                break;
             default:
-                log.error("Unknown live type error for {}", sw.getStringId());
+                // Error Live Type
+                log.error("checkAndMoveLiveFlowInternal, Unknown Live Type error!"
+                            + " AdaptiveStats collection thread for {}",
+                            sw.getStringId());
                 return false;
-            }
+        }
 
-            if (log.isTraceEnabled()) {
-                log.trace(CHECK_AND_MOVE_LOG, fe.id(), fe.state(), fe.flowLiveType(),
+        if (log.isTraceEnabled()) {
+            log.trace(CHECK_AND_MOVE_LOG, fe.id(), fe.state(), fe.liveType(),
                         liveTime, fe.life(), fe.bytes(), fe.packets(), fromLastSeen,
                         fe.priority(), fe.selector().criteria(), fe.treatment(),
                         sw.getStringId());
-            }
-
-            return true;
         }
 
-        /**
-         * Checks and moves live type for all type flow entries in table at every calAndPollInterval time.
-         */
-        public void checkAndMoveLiveFlowAll() {
-            Set<TypedStoredFlowEntry> typedFlowEntries = getFlowEntriesInternal();
-
-            long calCurTime = System.currentTimeMillis();
-            typedFlowEntries.forEach(fe -> {
-                if (!checkAndMoveLiveFlowInternal(fe, calCurTime)) {
-                    remove(fe);
-                }
-            });
-
-            // print table counts for debug
-            if (log.isTraceEnabled()) {
-                synchronized (this) {
-                    long totalFlowCount = getFlowCount();
-                    long shortFlowCount = shortFlows.size();
-                    long midFlowCount = midFlows.size();
-                    long longFlowCount = longFlows.size();
-                    long immediateFlowCount = totalFlowCount - shortFlowCount - midFlowCount - longFlowCount;
-                    long calTotalCount = addCount + addWithSetFlowLiveTypeCount - removeCount;
-
-                    log.trace(CHECK_AND_MOVE_COUNT_LOG, totalFlowCount, calTotalCount,
-                            immediateFlowCount, shortFlowCount, midFlowCount, longFlowCount,
-                            addCount, addWithSetFlowLiveTypeCount, removeCount, sw.getStringId());
-
-                    if (totalFlowCount != calTotalCount) {
-                        log.error("Real total flow count and calculated total flow count do NOT match");
-                    }
-                    if (immediateFlowCount < 0) {
-                        log.error("Immediate flow count is negative");
-                    }
-                }
-            }
-            log.trace("checkAndMoveLiveFlowAll: adaptiveStats for {}", sw.getStringId());
-        }
-
-        /**
-         * Removes the typed flow entry from table.
-         *
-         * @param rule the flow rule
-         */
-        public synchronized void remove(FlowRule rule) {
-            checkNotNull(rule);
-
-            TypedStoredFlowEntry removeStore = getFlowEntryInternal(rule);
-            if (removeStore != null) {
-                removeLiveFlowsInternal(removeStore);
-                boolean result = getFlowEntriesInternal(rule.id()).remove(removeStore);
-
-                if (result) {
-                    removeCount++;
-                }
-            }
-        }
-
-        // Remove the typed flow entry from corresponding table
-        private void removeLiveFlowsInternal(TypedStoredFlowEntry fe) {
-            switch (fe.flowLiveType()) {
-            case IMMEDIATE_FLOW:
-                // do nothing
-                break;
-            case SHORT_FLOW:
-                shortFlows.remove(fe);
-                break;
-            case MID_FLOW:
-                midFlows.remove(fe);
-                break;
-            case LONG_FLOW:
-                longFlows.remove(fe);
-                break;
-            default: // error in Flow Live Type
-                log.error("removeLiveFlowsInternal: unknown live type error");
-                break;
-            }
-        }
+        return true;
     }
 }
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java
index 9c9f154..886fb97 100644
--- a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/impl/OpenFlowRuleProvider.java
@@ -140,7 +140,9 @@
 
     private Cache<Long, InternalCacheEntry> pendingBatches;
 
-    private final Timer timer = new Timer("onos-openflow-flowstats-collector");
+    private final Timer timer = new Timer("onos-openflow-collector");
+
+    // Old simple collector set
     private final Map<Dpid, FlowStatsCollector> simpleCollectors = Maps.newConcurrentMap();
 
     // NewAdaptiveFlowStatsCollector Set
@@ -303,14 +305,6 @@
         }
         sw.sendMsg(FlowModBuilder.builder(flowRule, sw.factory(),
                 Optional.empty(), Optional.of(driverService)).buildFlowAdd());
-
-        if (adaptiveFlowSampling) {
-            // Add TypedFlowEntry to deviceFlowEntries in NewAdaptiveFlowStatsCollector
-            NewAdaptiveFlowStatsCollector collector = afsCollectors.get(dpid);
-            if (collector != null) {
-                collector.addWithFlowRule(flowRule);
-            }
-        }
     }
 
     @Override
@@ -336,14 +330,6 @@
         }
         sw.sendMsg(FlowModBuilder.builder(flowRule, sw.factory(),
                                           Optional.empty(), Optional.of(driverService)).buildFlowDel());
-
-        if (adaptiveFlowSampling) {
-            // Remove TypedFlowEntry to deviceFlowEntries in NewAdaptiveFlowStatsCollector
-            NewAdaptiveFlowStatsCollector collector = afsCollectors.get(dpid);
-            if (collector != null) {
-                collector.removeFlows(flowRule);
-            }
-        }
     }
 
     @Override
@@ -384,25 +370,12 @@
             switch (fbe.operator()) {
                 case ADD:
                     mod = builder.buildFlowAdd();
-                    if (adaptiveFlowSampling && collector != null) {
-                        // Add TypedFlowEntry to deviceFlowEntries in NewAdaptiveFlowStatsCollector
-                        collector.addWithFlowRule(fbe.target());
-                    }
                     break;
                 case REMOVE:
                     mod = builder.buildFlowDel();
-                    if (adaptiveFlowSampling && collector != null) {
-                        // Remove TypedFlowEntry to deviceFlowEntries in NewAdaptiveFlowStatsCollector
-                        collector.removeFlows(fbe.target());
-                    }
                     break;
                 case MODIFY:
                     mod = builder.buildFlowMod();
-                    if (adaptiveFlowSampling && collector != null) {
-                        // Add or Update TypedFlowEntry to deviceFlowEntries in NewAdaptiveFlowStatsCollector
-                        // afsCollectors.get(dpid).addWithFlowRule(fbe.target()); //check if add is good or not
-                        collector.addOrUpdateFlows((FlowEntry) fbe.target());
-                    }
                     break;
                 default:
                     log.error("Unsupported batch operation {}; skipping flowmod {}",
@@ -462,14 +435,6 @@
 
                     FlowEntry fr = new FlowEntryBuilder(deviceId, removed, driverService).build();
                     providerService.flowRemoved(fr);
-
-                    if (adaptiveFlowSampling) {
-                        // Removed TypedFlowEntry to deviceFlowEntries in NewAdaptiveFlowStatsCollector
-                        NewAdaptiveFlowStatsCollector collector = afsCollectors.get(dpid);
-                        if (collector != null) {
-                            collector.flowRemoved(fr);
-                        }
-                    }
                     break;
                 case STATS_REPLY:
                     if (((OFStatsReply) msg).getStatsType() == OFStatsType.FLOW) {
@@ -646,40 +611,34 @@
         private void pushFlowMetrics(Dpid dpid, OFFlowStatsReply replies) {
 
             DeviceId did = DeviceId.deviceId(Dpid.uri(dpid));
+            NewAdaptiveFlowStatsCollector afsc = afsCollectors.get(dpid);
 
-            List<FlowEntry> flowEntries = replies.getEntries().stream()
-                    .map(entry -> new FlowEntryBuilder(did, entry, driverService).build())
-                    .collect(Collectors.toList());
+            if (adaptiveFlowSampling && afsc != null)  {
+                List<FlowEntry> flowEntries = replies.getEntries().stream()
+                        .map(entry -> new FlowEntryBuilder(did, entry, driverService).withSetAfsc(afsc).build())
+                        .collect(Collectors.toList());
 
-            if (adaptiveFlowSampling)  {
-                NewAdaptiveFlowStatsCollector afsc = afsCollectors.get(dpid);
-
-                synchronized (afsc) {
-                    if (afsc.getFlowMissingXid() != NewAdaptiveFlowStatsCollector.NO_FLOW_MISSING_XID) {
-                        log.debug("OpenFlowRuleProvider:pushFlowMetrics, flowMissingXid={}, " +
-                                        "OFFlowStatsReply Xid={}, for {}",
-                                afsc.getFlowMissingXid(), replies.getXid(), dpid);
+                // Check that OFFlowStatsReply Xid is same with the one of OFFlowStatsRequest?
+                if (afsc.getFlowMissingXid() != NewAdaptiveFlowStatsCollector.NO_FLOW_MISSING_XID) {
+                        log.debug("OpenFlowRuleProvider:pushFlowMetrics, flowMissingXid={}, "
+                                          + "OFFlowStatsReply Xid={}, for {}",
+                                  afsc.getFlowMissingXid(), replies.getXid(), dpid);
+                    if (afsc.getFlowMissingXid() == replies.getXid()) {
+                        // call entire flow stats update with flowMissing synchronization.
+                        // used existing pushFlowMetrics
+                        providerService.pushFlowMetrics(did, flowEntries);
                     }
-
-                    // Check that OFFlowStatsReply Xid is same with the one of OFFlowStatsRequest?
-                    if (afsc.getFlowMissingXid() != NewAdaptiveFlowStatsCollector.NO_FLOW_MISSING_XID) {
-                        if (afsc.getFlowMissingXid() == replies.getXid()) {
-                            // call entire flow stats update with flowMissing synchronization.
-                            // used existing pushFlowMetrics
-                            providerService.pushFlowMetrics(did, flowEntries);
-                        }
-                        // reset flowMissingXid to NO_FLOW_MISSING_XID
-                        afsc.setFlowMissingXid(NewAdaptiveFlowStatsCollector.NO_FLOW_MISSING_XID);
-
-                    } else {
-                        // call individual flow stats update
-                        providerService.pushFlowMetricsWithoutFlowMissing(did, flowEntries);
-                    }
-
-                    // Update TypedFlowEntry to deviceFlowEntries in NewAdaptiveFlowStatsCollector
-                    afsc.pushFlowMetrics(flowEntries);
+                    // reset flowMissingXid to NO_FLOW_MISSING_XID
+                    afsc.setFlowMissingXid(NewAdaptiveFlowStatsCollector.NO_FLOW_MISSING_XID);
+                } else {
+                    // call individual flow stats update
+                    providerService.pushFlowMetricsWithoutFlowMissing(did, flowEntries);
                 }
             } else {
+                List<FlowEntry> flowEntries = replies.getEntries().stream()
+                        .map(entry -> new FlowEntryBuilder(did, entry, driverService).build())
+                        .collect(Collectors.toList());
+
                 // call existing entire flow stats update with flowMissing synchronization
                 providerService.pushFlowMetrics(did, flowEntries);
             }
diff --git a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java
index 2b042e6..c9f5dd3 100644
--- a/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java
+++ b/providers/openflow/flow/src/main/java/org/onosproject/provider/of/flow/util/FlowEntryBuilder.java
@@ -48,6 +48,7 @@
 import org.onosproject.net.flow.instructions.Instructions;
 import org.onosproject.openflow.controller.ExtensionSelectorInterpreter;
 import org.onosproject.openflow.controller.ExtensionTreatmentInterpreter;
+import org.onosproject.provider.of.flow.impl.NewAdaptiveFlowStatsCollector;
 import org.projectfloodlight.openflow.protocol.OFFlowMod;
 import org.projectfloodlight.openflow.protocol.OFFlowRemoved;
 import org.projectfloodlight.openflow.protocol.OFFlowStatsEntry;
@@ -124,6 +125,10 @@
 
     private final DriverService driverService;
 
+    // NewAdaptiveFlowStatsCollector for AdaptiveFlowSampling mode,
+    // null is not AFM mode, namely SimpleStatsCollector mode
+    private NewAdaptiveFlowStatsCollector afsc;
+
     public FlowEntryBuilder(DeviceId deviceId, OFFlowStatsEntry entry, DriverService driverService) {
         this.stat = entry;
         this.match = entry.getMatch();
@@ -133,6 +138,7 @@
         this.flowMod = null;
         this.type = FlowType.STAT;
         this.driverService = driverService;
+        this.afsc = null;
     }
 
     public FlowEntryBuilder(DeviceId deviceId, OFFlowRemoved removed, DriverService driverService) {
@@ -144,6 +150,7 @@
         this.flowMod = null;
         this.type = FlowType.REMOVED;
         this.driverService = driverService;
+        this.afsc = null;
     }
 
     public FlowEntryBuilder(DeviceId deviceId, OFFlowMod fm, DriverService driverService) {
@@ -155,6 +162,12 @@
         this.stat = null;
         this.removed = null;
         this.driverService = driverService;
+        this.afsc = null;
+    }
+
+    public FlowEntryBuilder withSetAfsc(NewAdaptiveFlowStatsCollector afsc) {
+        this.afsc = afsc;
+        return this;
     }
 
     public FlowEntry build(FlowEntryState... state) {
@@ -173,11 +186,21 @@
                         builder.forTable(stat.getTableId().getValue());
                     }
 
-                    return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
-                                                SECONDS.toNanos(stat.getDurationSec())
-                                                        + stat.getDurationNsec(), NANOSECONDS,
-                                                stat.getPacketCount().getValue(),
-                                                stat.getByteCount().getValue());
+                    if (afsc != null) {
+                        FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(stat.getDurationSec());
+                        return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                                                    SECONDS.toNanos(stat.getDurationSec())
+                                                            + stat.getDurationNsec(), NANOSECONDS,
+                                                    liveType,
+                                                    stat.getPacketCount().getValue(),
+                                                    stat.getByteCount().getValue());
+                    } else {
+                        return new DefaultFlowEntry(builder.build(), FlowEntryState.ADDED,
+                                                    stat.getDurationSec(),
+                                                    stat.getPacketCount().getValue(),
+                                                    stat.getByteCount().getValue());
+                    }
+
                 case REMOVED:
                     builder = DefaultFlowRule.builder()
                             .forDevice(deviceId)
@@ -191,11 +214,21 @@
                         builder.forTable(removed.getTableId().getValue());
                     }
 
-                    return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
-                                                SECONDS.toNanos(removed.getDurationSec())
-                                                        + removed.getDurationNsec(), NANOSECONDS,
-                                                removed.getPacketCount().getValue(),
-                                                removed.getByteCount().getValue());
+                    if (afsc != null) {
+                        FlowEntry.FlowLiveType liveType = afsc.calFlowLiveType(removed.getDurationSec());
+                        return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
+                                                    SECONDS.toNanos(removed.getDurationSec())
+                                                            + removed.getDurationNsec(), NANOSECONDS,
+                                                    liveType,
+                                                    removed.getPacketCount().getValue(),
+                                                    removed.getByteCount().getValue());
+                    } else {
+                        return new DefaultFlowEntry(builder.build(), FlowEntryState.REMOVED,
+                                                    removed.getDurationSec(),
+                                                    removed.getPacketCount().getValue(),
+                                                    removed.getByteCount().getValue());
+                    }
+
                 case MOD:
                     FlowEntryState flowState = state.length > 0 ? state[0] : FlowEntryState.FAILED;
                     builder = DefaultFlowRule.builder()
@@ -209,7 +242,12 @@
                         builder.forTable(flowMod.getTableId().getValue());
                     }
 
-                    return new DefaultFlowEntry(builder.build(), flowState, 0, 0, 0);
+                    if (afsc != null) {
+                        FlowEntry.FlowLiveType liveType = FlowEntry.FlowLiveType.IMMEDIATE;
+                        return new DefaultFlowEntry(builder.build(), flowState, 0, liveType, 0, 0);
+                    } else {
+                        return new DefaultFlowEntry(builder.build(), flowState, 0, 0, 0);
+                    }
                 default:
                     log.error("Unknown flow type : {}", this.type);
                     return null;