Brian O'Connor
Committed by Gerrit Code Review

ONOS-3023 Changing flowTable sets to map so that we can compare

stored vs. new rule when adding and removing

Change-Id: Ibd885023d550af3b2220056fbdf44ad8ec7fefda
......@@ -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 @@ public class DefaultFlowRule implements FlowRule {
return tableId;
}
@Beta
public long created() {
return created;
}
public static Builder builder() {
return new Builder();
}
......
......@@ -220,6 +220,7 @@ public class SimpleFlowRuleStore
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());
......
......@@ -311,6 +311,7 @@ public class FlowRuleManager
} 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 @@ public class FlowRuleManager
log.debug("Flow {} removed", flowRule);
post(event);
}
}
private void extraneousFlow(FlowRule flowRule) {
checkNotNull(flowRule, FLOW_RULE_NULL);
checkValidity();
......@@ -335,13 +334,11 @@ public class FlowRuleManager
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 @@ public class FlowRuleManager
log.debug("Removing flow rules....");
removeFlowRules(flowEntry);
}
}
private boolean checkRuleLiveness(FlowEntry swRule, FlowEntry storedRule) {
......
......@@ -269,23 +269,19 @@ public class FlowRuleManagerTest {
@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 @@ public class FlowRuleManagerTest {
providerService.flowRemoved(fe3);
validateEvents();
}
@Test
......
......@@ -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 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.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 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 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 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 @@ public class NewDistributedFlowRuleStore
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 @@ public class NewDistributedFlowRuleStore
// 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 @@ public class NewDistributedFlowRuleStore
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 @@ public class NewDistributedFlowRuleStore
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,11 +697,13 @@ public class NewDistributedFlowRuleStore
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,
clusterCommunicator.<Map<DeviceId,
Map<FlowId, Map<StoredFlowEntry, StoredFlowEntry>>>,
Set<DeviceId>>
sendAndReceive(deviceFlowEntries,
FLOW_TABLE_BACKUP,
SERIALIZER::encode,
SERIALIZER::decode,
......@@ -724,10 +731,10 @@ public class NewDistributedFlowRuleStore
* @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 @@ public class NewDistributedFlowRuleStore
}
}
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 class NewDistributedFlowRuleStore
}
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 @@ public class NewDistributedFlowRuleStore
}
}
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);
......
......@@ -416,6 +416,7 @@ public class NewAdaptiveFlowStatsCollector {
+ " 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 @@ public class NewAdaptiveFlowStatsCollector {
", 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());
......