Jonathan Hart
Committed by Brian O'Connor

Finished implementation of GossipIntentStore based on new API and semantics.

Change-Id: I1a71d075e5d34ab7b9f7c2533d389235d6da1d9a
......@@ -48,7 +48,7 @@ public class Key {
return new LongKey(key, appId);
}
private static final class StringKey extends Key {
public static final class StringKey extends Key {
private final ApplicationId appId;
private final String key;
......@@ -87,7 +87,7 @@ public class Key {
}
}
private static final class LongKey extends Key {
public static final class LongKey extends Key {
private final ApplicationId appId;
private final long key;
......
......@@ -199,7 +199,7 @@ public class HazelcastLeadershipService implements LeadershipService,
LeadershipEvent leadershipEvent =
SERIALIZER.decode(message.getMessageObject());
log.debug("Leadership Event: time = {} type = {} event = {}",
log.trace("Leadership Event: time = {} type = {} event = {}",
leadershipEvent.time(), leadershipEvent.type(),
leadershipEvent);
......
......@@ -18,15 +18,18 @@ package org.onosproject.store.impl;
import org.onosproject.store.Timestamp;
/**
* Clock service that can generate timestamps per object.
* Clock service that can generate timestamps based off of two input objects.
* Implementations are free to only take one or none of the objects into account
* when generating timestamps.
*/
public interface ClockService<T> {
public interface ClockService<T, U> {
/**
* Gets a new timestamp for the given object.
* Gets a new timestamp for the given objects.
*
* @param object Object to get a timestamp for
* @param object1 First object to use when generating timestamps
* @param object2 Second object to use when generating timestamps
* @return the new timestamp
*/
public Timestamp getTimestamp(T object);
public Timestamp getTimestamp(T object1, U object2);
}
......
......@@ -111,6 +111,27 @@ public interface EventuallyConsistentMap<K, V> {
public void remove(K key);
/**
* Removes the given key-value mapping from the map, if it exists.
* <p>
* This actually means remove any values up to and including the timestamp
* given by {@link org.onosproject.store.impl.ClockService#getTimestamp(Object, Object)}.
* Any mappings that produce an earlier timestamp than this given key-value
* pair will be removed, and any mappings that produce a later timestamp
* will supersede this remove.
* </p><p>
* Note: this differs from the specification of {@link java.util.Map}
* because it does not return a boolean indication whether a value was removed.
* Clients are expected to register an
* {@link org.onosproject.store.impl.EventuallyConsistentMapListener} if
* they are interested in receiving notification of updates to the map.
* </p>
*
* @param key the key to remove the mapping for
* @param value the value mapped to the key
*/
public void remove(K key, V value);
/**
* Adds mappings for all key-value pairs in the specified map to this map.
* <p>
* This will be more efficient in communication than calling individual put
......
......@@ -69,7 +69,7 @@ public class EventuallyConsistentMapImpl<K, V>
private final ClusterCommunicationService clusterCommunicator;
private final KryoSerializer serializer;
private final ClockService<K> clockService;
private final ClockService<K, V> clockService;
private final MessageSubject updateMessageSubject;
private final MessageSubject removeMessageSubject;
......@@ -126,7 +126,7 @@ public class EventuallyConsistentMapImpl<K, V>
ClusterService clusterService,
ClusterCommunicationService clusterCommunicator,
KryoNamespace.Builder serializerBuilder,
ClockService<K> clockService) {
ClockService<K, V> clockService) {
this.mapName = checkNotNull(mapName);
this.clusterService = checkNotNull(clusterService);
......@@ -227,7 +227,8 @@ public class EventuallyConsistentMapImpl<K, V>
checkNotNull(key, ERROR_NULL_KEY);
checkNotNull(value, ERROR_NULL_VALUE);
Timestamp timestamp = clockService.getTimestamp(key);
Timestamp timestamp = clockService.getTimestamp(key, value);
if (putInternal(key, value, timestamp)) {
notifyPeers(new InternalPutEvent<>(key, value, timestamp));
EventuallyConsistentMapEvent<K, V> externalEvent
......@@ -260,7 +261,9 @@ public class EventuallyConsistentMapImpl<K, V>
checkState(!destroyed, mapName + ERROR_DESTROYED);
checkNotNull(key, ERROR_NULL_KEY);
Timestamp timestamp = clockService.getTimestamp(key);
// TODO prevent calls here if value is important for timestamp
Timestamp timestamp = clockService.getTimestamp(key, null);
if (removeInternal(key, timestamp)) {
notifyPeers(new InternalRemoveEvent<>(key, timestamp));
EventuallyConsistentMapEvent<K, V> externalEvent
......@@ -283,6 +286,23 @@ public class EventuallyConsistentMapImpl<K, V>
}
@Override
public void remove(K key, V value) {
checkState(!destroyed, mapName + ERROR_DESTROYED);
checkNotNull(key, ERROR_NULL_KEY);
checkNotNull(value, ERROR_NULL_VALUE);
Timestamp timestamp = clockService.getTimestamp(key, value);
if (removeInternal(key, timestamp)) {
notifyPeers(new InternalRemoveEvent<>(key, timestamp));
EventuallyConsistentMapEvent<K, V> externalEvent
= new EventuallyConsistentMapEvent<>(
EventuallyConsistentMapEvent.Type.REMOVE, key, value);
notifyListeners(externalEvent);
}
}
@Override
public void putAll(Map<? extends K, ? extends V> m) {
checkState(!destroyed, mapName + ERROR_DESTROYED);
......@@ -295,7 +315,7 @@ public class EventuallyConsistentMapImpl<K, V>
checkNotNull(key, ERROR_NULL_KEY);
checkNotNull(value, ERROR_NULL_VALUE);
Timestamp timestamp = clockService.getTimestamp(entry.getKey());
Timestamp timestamp = clockService.getTimestamp(key, value);
if (putInternal(key, value, timestamp)) {
updates.add(new PutEntry<>(key, value, timestamp));
......@@ -306,7 +326,8 @@ public class EventuallyConsistentMapImpl<K, V>
notifyPeers(new InternalPutEvent<>(updates));
for (PutEntry<K, V> entry : updates) {
EventuallyConsistentMapEvent<K, V> externalEvent = new EventuallyConsistentMapEvent<>(
EventuallyConsistentMapEvent<K, V> externalEvent =
new EventuallyConsistentMapEvent<>(
EventuallyConsistentMapEvent.Type.PUT, entry.key(),
entry.value());
notifyListeners(externalEvent);
......@@ -321,7 +342,8 @@ public class EventuallyConsistentMapImpl<K, V>
List<RemoveEntry<K>> removed = new ArrayList<>(items.size());
for (K key : items.keySet()) {
Timestamp timestamp = clockService.getTimestamp(key);
// TODO also this is not applicable if value is important for timestamp?
Timestamp timestamp = clockService.getTimestamp(key, null);
if (removeInternal(key, timestamp)) {
removed.add(new RemoveEntry<>(key, timestamp));
......@@ -565,7 +587,8 @@ public class EventuallyConsistentMapImpl<K, V>
// Send all updates to the peer at once
if (!updatesToSend.isEmpty()) {
try {
unicastMessage(sender, updateMessageSubject, new InternalPutEvent<>(updatesToSend));
unicastMessage(sender, updateMessageSubject,
new InternalPutEvent<>(updatesToSend));
} catch (IOException e) {
log.warn("Failed to send advertisement response", e);
}
......@@ -603,7 +626,8 @@ public class EventuallyConsistentMapImpl<K, V>
// Send all removes to the peer at once
if (!removesToSend.isEmpty()) {
try {
unicastMessage(sender, removeMessageSubject, new InternalRemoveEvent<>(removesToSend));
unicastMessage(sender, removeMessageSubject,
new InternalRemoveEvent<>(removesToSend));
} catch (IOException e) {
log.warn("Failed to send advertisement response", e);
}
......
/*
* Copyright 2015 Open Networking Laboratory
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.impl;
import com.google.common.base.MoreObjects;
import com.google.common.collect.ComparisonChain;
import org.onosproject.store.Timestamp;
import java.util.Objects;
import static com.google.common.base.Preconditions.checkArgument;
/**
* A logical timestamp that derives its value from two input values. Value1
* always takes precedence over value2 when comparing timestamps.
*/
public class MultiValuedTimestamp implements Timestamp {
private final Timestamp timestamp;
private final long value2;
/**
* Creates a new timestamp based on two values. The first value has higher
* precedence than the second when comparing timestamps.
*
* @param timestamp first value
* @param value2 second value
*/
public MultiValuedTimestamp(Timestamp timestamp, long value2) {
this.timestamp = timestamp;
this.value2 = value2;
}
@Override
public int compareTo(Timestamp o) {
checkArgument(o instanceof MultiValuedTimestamp,
"Must be MultiValuedTimestamp", o);
MultiValuedTimestamp that = (MultiValuedTimestamp) o;
return ComparisonChain.start()
.compare(this.timestamp, that.timestamp)
.compare(this.value2, that.value2)
.result();
}
@Override
public int hashCode() {
return Objects.hash(timestamp, value2);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (!(obj instanceof MultiValuedTimestamp)) {
return false;
}
MultiValuedTimestamp that = (MultiValuedTimestamp) obj;
return Objects.equals(this.timestamp, that.timestamp) &&
Objects.equals(this.value2, that.value2);
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("timestamp", timestamp)
.add("value2", value2)
.toString();
}
/**
* Returns the first value.
*
* @return first value
*/
public Timestamp timestamp() {
return timestamp;
}
/**
* Returns the second value.
*
* @return second value
*/
public long sequenceNumber() {
return value2;
}
// Default constructor for serialization
@SuppressWarnings("unused")
private MultiValuedTimestamp() {
this.timestamp = null;
this.value2 = -1;
}
}
......@@ -20,9 +20,9 @@ import org.onosproject.store.Timestamp;
/**
* A clock service which hands out wallclock-based timestamps.
*/
public class WallclockClockManager<T> implements ClockService<T> {
public class WallclockClockManager<T, U> implements ClockService<T, U> {
@Override
public Timestamp getTimestamp(T object) {
public Timestamp getTimestamp(T object1, U object2) {
return new WallClockTimestamp();
}
}
......
......@@ -37,7 +37,8 @@ import org.onosproject.store.impl.EventuallyConsistentMap;
import org.onosproject.store.impl.EventuallyConsistentMapEvent;
import org.onosproject.store.impl.EventuallyConsistentMapImpl;
import org.onosproject.store.impl.EventuallyConsistentMapListener;
import org.onosproject.store.impl.WallclockClockManager;
import org.onosproject.store.impl.MultiValuedTimestamp;
import org.onosproject.store.impl.SystemClockTimestamp;
import org.onosproject.store.serializers.KryoNamespaces;
import org.slf4j.Logger;
......@@ -58,12 +59,6 @@ public class GossipIntentStore
private final Logger log = getLogger(getClass());
/*private EventuallyConsistentMap<IntentId, Intent> intents;
private EventuallyConsistentMap<IntentId, IntentState> intentStates;
private EventuallyConsistentMap<IntentId, List<Intent>> installables;*/
// Map of intent key => current intent state
private EventuallyConsistentMap<Key, IntentData> currentState;
......@@ -82,36 +77,22 @@ public class GossipIntentStore
@Activate
public void activate() {
KryoNamespace.Builder intentSerializer = KryoNamespace.newBuilder()
.register(KryoNamespaces.API);
/*intents = new EventuallyConsistentMapImpl<>("intents", clusterService,
clusterCommunicator,
intentSerializer,
new WallclockClockManager<>());
intentStates = new EventuallyConsistentMapImpl<>("intent-states",
clusterService,
clusterCommunicator,
intentSerializer,
new WallclockClockManager<>());
installables = new EventuallyConsistentMapImpl<>("intent-installables",
clusterService,
clusterCommunicator,
intentSerializer,
new WallclockClockManager<>());
*/
.register(KryoNamespaces.API)
.register(IntentData.class)
.register(MultiValuedTimestamp.class)
.register(SystemClockTimestamp.class);
currentState = new EventuallyConsistentMapImpl<>("intent-current",
clusterService,
clusterCommunicator,
intentSerializer,
new WallclockClockManager<>());
new IntentDataLogicalClockManager<>());
pending = new EventuallyConsistentMapImpl<>("intent-pending",
clusterService,
clusterCommunicator,
intentSerializer, // TODO
new WallclockClockManager<>());
new IntentDataClockManager<>());
currentState.addListener(new InternalIntentStatesListener());
pending.addListener(new InternalPendingListener());
......@@ -121,10 +102,6 @@ public class GossipIntentStore
@Deactivate
public void deactivate() {
/*intents.destroy();
intentStates.destroy();
installables.destroy();*/
currentState.destroy();
pending.destroy();
......@@ -133,7 +110,6 @@ public class GossipIntentStore
@Override
public long getIntentCount() {
//return intents.size();
return currentState.size();
}
......@@ -146,99 +122,45 @@ public class GossipIntentStore
@Override
public IntentState getIntentState(Key intentKey) {
// TODO: implement this
return IntentState.FAILED;
IntentData data = currentState.get(intentKey);
if (data != null) {
return data.state();
}
return null;
}
@Override
public List<Intent> getInstallableIntents(Key intentKey) {
// TODO: implement this or delete class
IntentData data = currentState.get(intentKey);
if (data != null) {
return data.installables();
}
return null;
/*
return installables.get(intentId);
*/
}
@Override
public List<BatchWrite.Operation> batchWrite(BatchWrite batch) {
/*
List<BatchWrite.Operation> failed = new ArrayList<>();
for (BatchWrite.Operation op : batch.operations()) {
switch (op.type()) {
case CREATE_INTENT:
checkArgument(op.args().size() == 1,
"CREATE_INTENT takes 1 argument. %s", op);
Intent intent = op.arg(0);
intents.put(intent.id(), intent);
intentStates.put(intent.id(), INSTALL_REQ);
// TODO remove from pending?
break;
case REMOVE_INTENT:
checkArgument(op.args().size() == 1,
"REMOVE_INTENT takes 1 argument. %s", op);
IntentId intentId = op.arg(0);
intents.remove(intentId);
intentStates.remove(intentId);
installables.remove(intentId);
break;
case SET_STATE:
checkArgument(op.args().size() == 2,
"SET_STATE takes 2 arguments. %s", op);
intent = op.arg(0);
IntentState newState = op.arg(1);
intentStates.put(intent.id(), newState);
break;
case SET_INSTALLABLE:
checkArgument(op.args().size() == 2,
"SET_INSTALLABLE takes 2 arguments. %s", op);
intentId = op.arg(0);
List<Intent> installableIntents = op.arg(1);
installables.put(intentId, installableIntents);
break;
case REMOVE_INSTALLED:
checkArgument(op.args().size() == 1,
"REMOVE_INSTALLED takes 1 argument. %s", op);
intentId = op.arg(0);
installables.remove(intentId);
break;
default:
log.warn("Unknown Operation encountered: {}", op);
failed.add(op);
break;
}
}
return failed;
*/
// Deprecated
return null;
}
@Override
public void write(IntentData newData) {
log.debug("writing intent {}", newData);
// Only the master is modifying the current state. Therefore assume
// this always succeeds
currentState.put(newData.key(), newData);
// if current.put succeeded
//pending.remove(newData.key(), newData);
pending.remove(newData.key(), newData);
try {
/*try {
notifyDelegate(IntentEvent.getEvent(newData));
} catch (IllegalArgumentException e) {
//no-op
log.trace("ignore this exception: {}", e);
}
}*/
}
@Override
......@@ -262,14 +184,17 @@ public class GossipIntentStore
@Override
public void addPending(IntentData data) {
log.debug("new call to pending {}", data);
if (data.version() == null) {
log.debug("updating timestamp");
data.setVersion(new SystemClockTimestamp());
}
pending.put(data.key(), data);
}
@Override
public boolean isMaster(Intent intent) {
// TODO
//return partitionService.isMine(intent.key());
return false;
return partitionService.isMine(intent.key());
}
private void notifyDelegateIfNotNull(IntentEvent event) {
......@@ -284,18 +209,16 @@ public class GossipIntentStore
public void event(
EventuallyConsistentMapEvent<Key, IntentData> event) {
if (event.type() == EventuallyConsistentMapEvent.Type.PUT) {
// TODO check event send logic
IntentEvent externalEvent;
IntentData intentData = currentState.get(event.key()); // TODO OK if this is null?
IntentData intentData = event.value();
/*
try {
externalEvent = IntentEvent.getEvent(event.value(), intent);
externalEvent = IntentEvent.getEvent(intentData.state(), intentData.intent());
} catch (IllegalArgumentException e) {
externalEvent = null;
}
notifyDelegateIfNotNull(externalEvent);*/
notifyDelegateIfNotNull(externalEvent);
}
}
}
......@@ -314,6 +237,13 @@ public class GossipIntentStore
delegate.process(event.value());
}
}
try {
notifyDelegate(IntentEvent.getEvent(event.value()));
} catch (IllegalArgumentException e) {
//no-op
log.trace("ignore this exception: {}", e);
}
}
}
}
......
......@@ -20,11 +20,11 @@ import org.onosproject.store.Timestamp;
import org.onosproject.store.impl.ClockService;
/**
* ClockService that generates timestamps based on IntentData versions.
* ClockService that uses IntentData versions as timestamps.
*/
public class IntentDataClockManager implements ClockService<IntentData> {
public class IntentDataClockManager<K> implements ClockService<K, IntentData> {
@Override
public Timestamp getTimestamp(IntentData data) {
return null;
public Timestamp getTimestamp(K key, IntentData intentData) {
return intentData.version();
}
}
......
/*
* Copyright 2015 Open Networking Laboratory
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.intent.impl;
import org.onosproject.net.intent.IntentData;
import org.onosproject.store.Timestamp;
import org.onosproject.store.impl.ClockService;
import org.onosproject.store.impl.MultiValuedTimestamp;
import java.util.concurrent.atomic.AtomicLong;
/**
* ClockService that generates logical timestamps based on IntentData versions.
*/
public class IntentDataLogicalClockManager<K> implements ClockService<K, IntentData> {
private final AtomicLong sequenceNumber = new AtomicLong(0);
@Override
public Timestamp getTimestamp(K key, IntentData intentData) {
return new MultiValuedTimestamp(intentData.version(), sequenceNumber.getAndIncrement());
}
}
......@@ -24,14 +24,14 @@ import java.util.Objects;
* processed by a single ONOS instance at a time.
*/
public class PartitionId {
private final int id;
private final long id;
/**
* Creates a new partition ID.
*
* @param id the partition ID
*/
PartitionId(int id) {
PartitionId(long id) {
this.id = id;
}
......
......@@ -26,6 +26,7 @@ import org.onosproject.cluster.Leadership;
import org.onosproject.cluster.LeadershipEvent;
import org.onosproject.cluster.LeadershipEventListener;
import org.onosproject.cluster.LeadershipService;
import org.onosproject.net.intent.Key;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -33,8 +34,6 @@ import java.util.Collections;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Manages the assignment of intent keyspace partitions to instances.
*/
......@@ -75,14 +74,15 @@ public class PartitionManager implements PartitionService {
leadershipService.removeListener(leaderListener);
}
private PartitionId getPartitionForKey(String intentKey) {
return new PartitionId(intentKey.hashCode() % NUM_PARTITIONS);
private PartitionId getPartitionForKey(Key intentKey) {
log.debug("Getting partition for {}: {}", intentKey,
new PartitionId(Math.abs(intentKey.hash()) % NUM_PARTITIONS));
return new PartitionId(Math.abs(intentKey.hash()) % NUM_PARTITIONS);
}
@Override
public boolean isMine(String intentKey) {
return checkNotNull(
myPartitions.contains(getPartitionForKey(intentKey)));
public boolean isMine(Key intentKey) {
return myPartitions.contains(getPartitionForKey(intentKey));
}
private final class InternalLeadershipListener implements LeadershipEventListener {
......@@ -115,7 +115,6 @@ public class PartitionManager implements PartitionService {
myPartitions.remove(new PartitionId(partitionId));
}
}
}
}
}
......
......@@ -15,6 +15,8 @@
*/
package org.onosproject.store.intent.impl;
import org.onosproject.net.intent.Key;
/**
* Service for interacting with the partition-to-instance assignments.
*/
......@@ -27,7 +29,7 @@ public interface PartitionService {
* @param intentKey intent key to query
* @return true if the key is owned by this instance, otherwise false
*/
boolean isMine(String intentKey);
boolean isMine(Key intentKey);
// TODO add API for rebalancing partitions
}
......
......@@ -62,7 +62,7 @@ public class EventuallyConsistentMapImplTest {
private ClusterService clusterService;
private ClusterCommunicationService clusterCommunicator;
private SequentialClockService<String> clockService;
private SequentialClockService<String, String> clockService;
private static final String MAP_NAME = "test";
private static final MessageSubject PUT_MESSAGE_SUBJECT
......@@ -222,7 +222,7 @@ public class EventuallyConsistentMapImplTest {
// Remote put
ClusterMessage message
= generatePutMessage(KEY2, VALUE2, clockService.getTimestamp(KEY2));
= generatePutMessage(KEY2, VALUE2, clockService.getTimestamp(KEY2, VALUE2));
// Create a latch so we know when the put operation has finished
latch = new CountDownLatch(1);
......@@ -240,7 +240,7 @@ public class EventuallyConsistentMapImplTest {
// Remote remove
ClusterMessage removeMessage
= generateRemoveMessage(KEY1, clockService.getTimestamp(KEY1));
= generateRemoveMessage(KEY1, clockService.getTimestamp(KEY1, VALUE1));
// Create a latch so we know when the remove operation has finished
latch = new CountDownLatch(1);
......@@ -731,14 +731,15 @@ public class EventuallyConsistentMapImplTest {
* to give out timestamps from the past.
*
* @param <T> Type that the clock service will give out timestamps for
* @param <U> Second type that the clock service will give out values for
*/
private class SequentialClockService<T> implements ClockService<T> {
private class SequentialClockService<T, U> implements ClockService<T, U> {
private static final long INITIAL_VALUE = 1;
private final AtomicLong counter = new AtomicLong(INITIAL_VALUE);
@Override
public Timestamp getTimestamp(T object) {
public Timestamp getTimestamp(T object, U object2) {
return new TestTimestamp(counter.getAndIncrement());
}
......@@ -748,7 +749,7 @@ public class EventuallyConsistentMapImplTest {
* still allowing the CUT to get the same timestamp.
*
* @return timestamp equal to the timestamp that will be returned by the
* next call to {@link #getTimestamp(T)}.
* next call to {@link #getTimestamp(T, U)}.
*/
public Timestamp peekAtNextTimestamp() {
return peek(1);
......
......@@ -85,6 +85,7 @@ import org.onosproject.net.intent.Intent;
import org.onosproject.net.intent.IntentId;
import org.onosproject.net.intent.IntentOperation;
import org.onosproject.net.intent.IntentState;
import org.onosproject.net.intent.Key;
import org.onosproject.net.intent.LinkCollectionIntent;
import org.onosproject.net.intent.MultiPointToSinglePointIntent;
import org.onosproject.net.intent.OpticalConnectivityIntent;
......@@ -285,6 +286,9 @@ public final class KryoNamespaces {
FlowRuleBatchEntry.FlowRuleOperation.class,
IntentId.class,
IntentState.class,
Key.class,
Key.LongKey.class,
Key.StringKey.class,
Intent.class,
ConnectivityIntent.class,
PathIntent.class,
......