Madan Jampani

Simplified ECMap implmentation by merging items and tombstones maps

Change-Id: If4253722d91c35a7e57dec3c2fceb216d14a7314
/*
* 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.ecmap;
import java.util.Objects;
import org.onosproject.store.Timestamp;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Base class for events in an EventuallyConsistentMap.
*/
public abstract class AbstractEntry<K, V> implements Comparable<AbstractEntry<K, V>> {
private final K key;
private final Timestamp timestamp;
/**
* Creates a new put entry.
*
* @param key key of the entry
* @param timestamp timestamp of the put event
*/
public AbstractEntry(K key, Timestamp timestamp) {
this.key = checkNotNull(key);
this.timestamp = checkNotNull(timestamp);
}
// Needed for serialization.
@SuppressWarnings("unused")
protected AbstractEntry() {
this.key = null;
this.timestamp = null;
}
/**
* Returns the key of the entry.
*
* @return the key
*/
public K key() {
return key;
}
/**
* Returns the timestamp of the event.
*
* @return the timestamp
*/
public Timestamp timestamp() {
return timestamp;
}
@Override
public int compareTo(AbstractEntry<K, V> o) {
return this.timestamp.compareTo(o.timestamp);
}
@Override
public int hashCode() {
return Objects.hash(timestamp);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o instanceof AbstractEntry) {
final AbstractEntry that = (AbstractEntry) o;
return this.timestamp.equals(that.timestamp);
}
return false;
}
}
......@@ -16,11 +16,11 @@
package org.onosproject.store.ecmap;
import com.google.common.base.MoreObjects;
import com.google.common.collect.ImmutableMap;
import org.onosproject.cluster.NodeId;
import org.onosproject.store.Timestamp;
import java.util.Map;
import static com.google.common.base.Preconditions.checkNotNull;
/**
......@@ -29,22 +29,18 @@ import static com.google.common.base.Preconditions.checkNotNull;
public class AntiEntropyAdvertisement<K> {
private final NodeId sender;
private final Map<K, Timestamp> timestamps;
private final Map<K, Timestamp> tombstones;
private final Map<K, MapValue.Digest> digest;
/**
* Creates a new anti entropy advertisement message.
*
* @param sender the sender's node ID
* @param timestamps map of item key to timestamp for current items
* @param tombstones map of item key to timestamp for removed items
* @param digest for map entries
*/
public AntiEntropyAdvertisement(NodeId sender,
Map<K, Timestamp> timestamps,
Map<K, Timestamp> tombstones) {
Map<K, MapValue.Digest> digest) {
this.sender = checkNotNull(sender);
this.timestamps = checkNotNull(timestamps);
this.tombstones = checkNotNull(tombstones);
this.digest = ImmutableMap.copyOf(checkNotNull(digest));
}
/**
......@@ -57,36 +53,19 @@ public class AntiEntropyAdvertisement<K> {
}
/**
* Returns the map of current item timestamps.
* Returns the digest for map entries.
*
* @return current item timestamps
* @return mapping from key to associated digest
*/
public Map<K, Timestamp> timestamps() {
return timestamps;
}
/**
* Returns the map of removed item timestamps.
*
* @return removed item timestamps
*/
public Map<K, Timestamp> tombstones() {
return tombstones;
}
// For serializer
@SuppressWarnings("unused")
private AntiEntropyAdvertisement() {
this.sender = null;
this.timestamps = null;
this.tombstones = null;
public Map<K, MapValue.Digest> digest() {
return digest;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("timestampsSize", timestamps.size())
.add("tombstonesSize", tombstones.size())
.add("sender", sender)
.add("totalEntries", digest.size())
.toString();
}
}
......
......@@ -16,13 +16,10 @@
package org.onosproject.store.ecmap;
import org.apache.commons.lang3.mutable.MutableBoolean;
import org.mapdb.DB;
import org.mapdb.DBMaker;
import org.mapdb.Hasher;
import org.mapdb.Serializer;
import org.onosproject.store.Timestamp;
import org.onosproject.store.impl.Timestamped;
import org.onosproject.store.serializers.KryoSerializer;
import java.io.File;
......@@ -42,7 +39,6 @@ class MapDbPersistentStore<K, V> implements PersistentStore<K, V> {
private final DB database;
private final Map<byte[], byte[]> items;
private final Map<byte[], byte[]> tombstones;
/**
* Creates a new MapDB based persistent store.
......@@ -65,102 +61,32 @@ class MapDbPersistentStore<K, V> implements PersistentStore<K, V> {
.valueSerializer(Serializer.BYTE_ARRAY)
.hasher(Hasher.BYTE_ARRAY)
.makeOrGet();
tombstones = database.createHashMap("tombstones")
.keySerializer(Serializer.BYTE_ARRAY)
.valueSerializer(Serializer.BYTE_ARRAY)
.hasher(Hasher.BYTE_ARRAY)
.makeOrGet();
}
@Override
public void readInto(Map<K, Timestamped<V>> items, Map<K, Timestamp> tombstones) {
public void readInto(Map<K, MapValue<V>> items) {
this.items.forEach((keyBytes, valueBytes) ->
items.put(serializer.decode(keyBytes),
serializer.decode(valueBytes)));
this.tombstones.forEach((keyBytes, valueBytes) ->
tombstones.put(serializer.decode(keyBytes),
serializer.decode(valueBytes)));
serializer.decode(valueBytes)));
}
@Override
public void put(K key, V value, Timestamp timestamp) {
executor.submit(() -> putInternal(key, value, timestamp));
public void update(K key, MapValue<V> value) {
executor.submit(() -> updateInternal(key, value));
}
private void putInternal(K key, V value, Timestamp timestamp) {
private void updateInternal(K key, MapValue<V> newValue) {
byte[] keyBytes = serializer.encode(key);
byte[] removedBytes = tombstones.get(keyBytes);
Timestamp removed = removedBytes == null ? null :
serializer.decode(removedBytes);
if (removed != null && removed.isNewerThan(timestamp)) {
return;
}
final MutableBoolean updated = new MutableBoolean(false);
items.compute(keyBytes, (k, existingBytes) -> {
Timestamped<V> existing = existingBytes == null ? null :
MapValue<V> existing = existingBytes == null ? null :
serializer.decode(existingBytes);
if (existing != null && existing.isNewerThan(timestamp)) {
updated.setFalse();
return existingBytes;
if (existing == null || newValue.isNewerThan(existing)) {
return serializer.encode(newValue);
} else {
updated.setTrue();
return serializer.encode(new Timestamped<>(value, timestamp));
}
});
boolean success = updated.booleanValue();
if (success && removed != null) {
tombstones.remove(keyBytes, removedBytes);
}
database.commit();
}
@Override
public void remove(K key, Timestamp timestamp) {
executor.submit(() -> removeInternal(key, timestamp));
}
private void removeInternal(K key, Timestamp timestamp) {
byte[] keyBytes = serializer.encode(key);
final MutableBoolean updated = new MutableBoolean(false);
items.compute(keyBytes, (k, existingBytes) -> {
Timestamp existing = existingBytes == null ? null :
serializer.decode(existingBytes);
if (existing != null && existing.isNewerThan(timestamp)) {
updated.setFalse();
return existingBytes;
} else {
updated.setTrue();
// remove from items map
return null;
}
});
if (!updated.booleanValue()) {
return;
}
byte[] timestampBytes = serializer.encode(timestamp);
byte[] removedBytes = tombstones.get(keyBytes);
Timestamp removedTimestamp = removedBytes == null ? null :
serializer.decode(removedBytes);
if (removedTimestamp == null) {
tombstones.putIfAbsent(keyBytes, timestampBytes);
} else if (timestamp.isNewerThan(removedTimestamp)) {
tombstones.replace(keyBytes, removedBytes, timestampBytes);
}
database.commit();
}
}
......
package org.onosproject.store.ecmap;
import org.onosproject.store.Timestamp;
import com.google.common.base.MoreObjects;
/**
* Representation of a value in EventuallyConsistentMap.
*
* @param <V> value type
*/
public class MapValue<V> implements Comparable<MapValue<V>> {
private final Timestamp timestamp;
private final V value;
public MapValue(V value, Timestamp timestamp) {
this.value = value;
this.timestamp = timestamp;
}
public boolean isTombstone() {
return value == null;
}
public boolean isAlive() {
return value != null;
}
public Timestamp timestamp() {
return timestamp;
}
public V get() {
return value;
}
@Override
public int compareTo(MapValue<V> o) {
return this.timestamp.compareTo(o.timestamp);
}
public boolean isNewerThan(MapValue<V> other) {
return timestamp.isNewerThan(other.timestamp);
}
public boolean isNewerThan(Timestamp timestamp) {
return timestamp.isNewerThan(timestamp);
}
public Digest digest() {
return new Digest(timestamp, isTombstone());
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("timestamp", timestamp)
.add("value", value)
.toString();
}
@SuppressWarnings("unused")
private MapValue() {
this.timestamp = null;
this.value = null;
}
/**
* Digest or summary of a MapValue for use during Anti-Entropy exchanges.
*/
public static class Digest {
private final Timestamp timestamp;
private final boolean isTombstone;
public Digest(Timestamp timestamp, boolean isTombstone) {
this.timestamp = timestamp;
this.isTombstone = isTombstone;
}
public Timestamp timestamp() {
return timestamp;
}
public boolean isTombstone() {
return isTombstone;
}
public boolean isNewerThan(Digest other) {
return timestamp.isNewerThan(other.timestamp);
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("timestamp", timestamp)
.add("isTombstone", isTombstone)
.toString();
}
}
}
......@@ -16,9 +16,6 @@
package org.onosproject.store.ecmap;
import org.onosproject.store.Timestamp;
import org.onosproject.store.impl.Timestamped;
import java.util.Map;
/**
......@@ -30,24 +27,14 @@ interface PersistentStore<K, V> {
* Read the contents of the disk into the given maps.
*
* @param items items map
* @param tombstones tombstones map
*/
void readInto(Map<K, Timestamped<V>> items, Map<K, Timestamp> tombstones);
void readInto(Map<K, MapValue<V>> items);
/**
* Puts a new key,value pair into the map on disk.
* Updates a key,value pair in the persistent store.
*
* @param key the key
* @param value the value
* @param timestamp the timestamp of the update
*/
void put(K key, V value, Timestamp timestamp);
/**
* Removes a key from the map on disk.
*
* @param key the key
* @param timestamp the timestamp of the update
*/
void remove(K key, Timestamp timestamp);
void update(K key, MapValue<V> value);
}
......
/*
* 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.ecmap;
import com.google.common.base.MoreObjects;
import org.onosproject.store.Timestamp;
/**
* Describes a single remove event in an EventuallyConsistentMap.
*/
final class RemoveEntry<K, V> extends AbstractEntry<K, V> {
/**
* Creates a new remove entry.
*
* @param key key of the entry
* @param timestamp timestamp of the remove event
*/
public RemoveEntry(K key, Timestamp timestamp) {
super(key, timestamp);
}
// Needed for serialization.
@SuppressWarnings("unused")
private RemoveEntry() {
super();
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("key", key())
.add("timestamp", timestamp())
.toString();
}
}
......@@ -15,34 +15,35 @@
*/
package org.onosproject.store.ecmap;
import com.google.common.base.MoreObjects;
import org.onosproject.store.Timestamp;
import static com.google.common.base.Preconditions.checkNotNull;
import com.google.common.base.MoreObjects;
/**
* Describes a single put event in an EventuallyConsistentMap.
* Describes a single update event in an EventuallyConsistentMap.
*/
final class PutEntry<K, V> extends AbstractEntry<K, V> {
private final V value;
final class UpdateEntry<K, V> implements Comparable<UpdateEntry<K, V>> {
private final K key;
private final MapValue<V> value;
/**
* Creates a new put entry.
* Creates a new update entry.
*
* @param key key of the entry
* @param value value of the entry
* @param timestamp timestamp of the put event
*/
public PutEntry(K key, V value, Timestamp timestamp) {
super(key, timestamp);
public UpdateEntry(K key, MapValue<V> value) {
this.key = checkNotNull(key);
this.value = checkNotNull(value);
}
// Needed for serialization.
@SuppressWarnings("unused")
private PutEntry() {
super();
this.value = null;
/**
* Returns the key.
*
* @return the key
*/
public K key() {
return key;
}
/**
......@@ -50,16 +51,26 @@ final class PutEntry<K, V> extends AbstractEntry<K, V> {
*
* @return the value
*/
public V value() {
public MapValue<V> value() {
return value;
}
@Override
public int compareTo(UpdateEntry<K, V> o) {
return this.value.timestamp().compareTo(o.value.timestamp());
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("key", key())
.add("value", value)
.add("timestamp", timestamp())
.toString();
}
@SuppressWarnings("unused")
private UpdateEntry() {
this.key = null;
this.value = null;
}
}
......
......@@ -16,8 +16,8 @@
package org.onosproject.store.ecmap;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.MoreExecutors;
import org.junit.After;
......@@ -32,7 +32,6 @@ import org.onosproject.cluster.NodeId;
import org.onosproject.event.AbstractEvent;
import org.onosproject.store.Timestamp;
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.cluster.messaging.MessageSubject;
import org.onosproject.store.impl.LogicalTimestamp;
......@@ -44,11 +43,13 @@ import org.onosproject.store.service.EventuallyConsistentMapEvent;
import org.onosproject.store.service.EventuallyConsistentMapListener;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
......@@ -89,8 +90,8 @@ public class EventuallyConsistentMapImplTest {
private final ControllerNode self =
new DefaultControllerNode(new NodeId("local"), IpAddress.valueOf(1));
private ClusterMessageHandler updateHandler;
private ClusterMessageHandler antiEntropyHandler;
private Consumer<Collection<UpdateEntry<String, String>>> updateHandler;
private Consumer<AntiEntropyAdvertisement<String>> antiEntropyHandler;
/*
* Serialization is a bit tricky here. We need to serialize in the tests
......@@ -109,11 +110,10 @@ public class EventuallyConsistentMapImplTest {
// Below is the classes that the map internally registers
.register(LogicalTimestamp.class)
.register(WallClockTimestamp.class)
.register(PutEntry.class)
.register(RemoveEntry.class)
.register(ArrayList.class)
.register(AntiEntropyAdvertisement.class)
.register(HashMap.class)
.register(Optional.class)
.build();
}
};
......@@ -131,9 +131,9 @@ public class EventuallyConsistentMapImplTest {
// delegate to our ClusterCommunicationService implementation. This
// allows us to get a reference to the map's internal cluster message
// handlers so we can induce events coming in from a peer.
clusterCommunicator.addSubscriber(anyObject(MessageSubject.class),
anyObject(ClusterMessageHandler.class), anyObject(ExecutorService.class));
expectLastCall().andDelegateTo(new TestClusterCommunicationService()).times(3);
clusterCommunicator.<String>addSubscriber(anyObject(MessageSubject.class),
anyObject(Function.class), anyObject(Consumer.class), anyObject(Executor.class));
expectLastCall().andDelegateTo(new TestClusterCommunicationService()).times(2);
replay(clusterCommunicator);
......@@ -237,15 +237,15 @@ public class EventuallyConsistentMapImplTest {
assertEquals(VALUE1, ecMap.get(KEY1));
// Remote put
ClusterMessage message
= generatePutMessage(KEY2, VALUE2, clockService.getTimestamp(KEY2, VALUE2));
List<UpdateEntry<String, String>> message
= ImmutableList.of(generatePutMessage(KEY2, VALUE2, clockService.getTimestamp(KEY2, VALUE2)));
// Create a latch so we know when the put operation has finished
latch = new CountDownLatch(1);
ecMap.addListener(new TestListener(latch));
assertNull(ecMap.get(KEY2));
updateHandler.handle(message);
updateHandler.accept(message);
assertTrue("External listener never got notified of internal event",
latch.await(100, TimeUnit.MILLISECONDS));
assertEquals(VALUE2, ecMap.get(KEY2));
......@@ -255,14 +255,13 @@ public class EventuallyConsistentMapImplTest {
assertNull(ecMap.get(KEY2));
// Remote remove
ClusterMessage removeMessage
= generateRemoveMessage(KEY1, clockService.getTimestamp(KEY1, VALUE1));
message = ImmutableList.of(generateRemoveMessage(KEY1, clockService.getTimestamp(KEY1, VALUE1)));
// Create a latch so we know when the remove operation has finished
latch = new CountDownLatch(1);
ecMap.addListener(new TestListener(latch));
updateHandler.handle(removeMessage);
updateHandler.accept(message);
assertTrue("External listener never got notified of internal event",
latch.await(100, TimeUnit.MILLISECONDS));
assertNull(ecMap.get(KEY1));
......@@ -601,49 +600,35 @@ public class EventuallyConsistentMapImplTest {
}
}
private ClusterMessage generatePutMessage(String key, String value, Timestamp timestamp) {
PutEntry<String, String> event = new PutEntry<>(key, value, timestamp);
return new ClusterMessage(
clusterService.getLocalNode().id(), UPDATE_MESSAGE_SUBJECT,
SERIALIZER.encode(Lists.newArrayList(event)));
private UpdateEntry<String, String> generatePutMessage(String key, String value, Timestamp timestamp) {
return new UpdateEntry<>(key, new MapValue<>(value, timestamp));
}
private List<PutEntry<String, String>> generatePutMessage(
private List<UpdateEntry<String, String>> generatePutMessage(
String key1, String value1, String key2, String value2) {
ArrayList<PutEntry<String, String>> list = new ArrayList<>();
List<UpdateEntry<String, String>> list = new ArrayList<>();
Timestamp timestamp1 = clockService.peek(1);
Timestamp timestamp2 = clockService.peek(2);
PutEntry<String, String> pe1 = new PutEntry<>(key1, value1, timestamp1);
PutEntry<String, String> pe2 = new PutEntry<>(key2, value2, timestamp2);
list.add(pe1);
list.add(pe2);
list.add(generatePutMessage(key1, value1, timestamp1));
list.add(generatePutMessage(key2, value2, timestamp2));
return list;
}
private ClusterMessage generateRemoveMessage(String key, Timestamp timestamp) {
RemoveEntry<String, String> event = new RemoveEntry<>(key, timestamp);
return new ClusterMessage(
clusterService.getLocalNode().id(), UPDATE_MESSAGE_SUBJECT,
SERIALIZER.encode(Lists.newArrayList(event)));
private UpdateEntry<String, String> generateRemoveMessage(String key, Timestamp timestamp) {
return new UpdateEntry<>(key, new MapValue<>(null, timestamp));
}
private List<RemoveEntry<String, String>> generateRemoveMessage(String key1, String key2) {
ArrayList<RemoveEntry<String, String>> list = new ArrayList<>();
private List<UpdateEntry<String, String>> generateRemoveMessage(String key1, String key2) {
List<UpdateEntry<String, String>> list = new ArrayList<>();
Timestamp timestamp1 = clockService.peek(1);
Timestamp timestamp2 = clockService.peek(2);
RemoveEntry<String, String> re1 = new RemoveEntry<>(key1, timestamp1);
RemoveEntry<String, String> re2 = new RemoveEntry<>(key2, timestamp2);
list.add(re1);
list.add(re2);
list.add(generateRemoveMessage(key1, timestamp1));
list.add(generateRemoveMessage(key2, timestamp2));
return list;
}
......@@ -737,13 +722,6 @@ public class EventuallyConsistentMapImplTest {
public void addSubscriber(MessageSubject subject,
ClusterMessageHandler subscriber,
ExecutorService executor) {
if (subject.equals(UPDATE_MESSAGE_SUBJECT)) {
updateHandler = subscriber;
} else if (subject.equals(ANTI_ENTROPY_MESSAGE_SUBJECT)) {
antiEntropyHandler = subscriber;
} else {
throw new RuntimeException("Unexpected message subject " + subject.toString());
}
}
@Override
......@@ -793,6 +771,13 @@ public class EventuallyConsistentMapImplTest {
public <M> void addSubscriber(MessageSubject subject,
Function<byte[], M> decoder, Consumer<M> handler,
Executor executor) {
if (subject.equals(UPDATE_MESSAGE_SUBJECT)) {
updateHandler = (Consumer<Collection<UpdateEntry<String, String>>>) handler;
} else if (subject.equals(ANTI_ENTROPY_MESSAGE_SUBJECT)) {
antiEntropyHandler = (Consumer<AntiEntropyAdvertisement<String>>) handler;
} else {
throw new RuntimeException("Unexpected message subject " + subject.toString());
}
}
}
......