Madan Jampani
Committed by Gerrit Code Review

Updates to ConsistentMap and LeaderElector state machines

Change-Id: I7734b253a56fef7300a8a094a3cfc8c1b45c2453
......@@ -15,6 +15,7 @@
*/
package org.onosproject.store.primitives.impl;
import static org.slf4j.LoggerFactory.getLogger;
import io.atomix.catalyst.serializer.Serializer;
import io.atomix.catalyst.transport.Address;
import io.atomix.catalyst.transport.Transport;
......@@ -35,6 +36,7 @@ import java.util.function.Supplier;
import org.onosproject.cluster.NodeId;
import org.onosproject.store.service.PartitionInfo;
import org.slf4j.Logger;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
......@@ -44,6 +46,8 @@ import com.google.common.collect.Sets;
*/
public class StoragePartitionServer implements Managed<StoragePartitionServer> {
private final Logger log = getLogger(getClass());
private static final int MAX_ENTRIES_PER_LOG_SEGMENT = 32768;
private final StoragePartition partition;
private final Address localAddress;
......@@ -81,7 +85,13 @@ public class StoragePartitionServer implements Managed<StoragePartitionServer> {
} else {
serverOpenFuture = CompletableFuture.completedFuture(null);
}
return serverOpenFuture.thenApply(v -> null);
return serverOpenFuture.whenComplete((r, e) -> {
if (e == null) {
log.info("Successfully started server for partition {}", partition.getId());
} else {
log.info("Failed to start server for partition {}", partition.getId(), e);
}
}).thenApply(v -> null);
}
@Override
......@@ -105,7 +115,6 @@ public class StoragePartitionServer implements Managed<StoragePartitionServer> {
.withStorage(Storage.builder()
// FIXME: StorageLevel should be DISK
.withStorageLevel(StorageLevel.MEMORY)
.withSerializer(serializer.clone())
.withDirectory(dataFolder)
.withMaxEntriesPerSegment(MAX_ENTRIES_PER_LOG_SEGMENT)
.build())
......
......@@ -17,12 +17,12 @@ package org.onosproject.store.primitives.resources.impl;
import io.atomix.catalyst.util.Listener;
import io.atomix.copycat.client.CopycatClient;
import io.atomix.resource.Consistency;
import io.atomix.resource.Resource;
import io.atomix.resource.ResourceTypeInfo;
import java.util.Collection;
import java.util.ConcurrentModificationException;
import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
......@@ -49,7 +49,7 @@ public class AtomixConsistentMap extends Resource<AtomixConsistentMap, Resource.
private final Set<MapEventListener<String, byte[]>> mapEventListeners = Sets.newCopyOnWriteArraySet();
private static final String CHANGE_SUBJECT = "change";
public static final String CHANGE_SUBJECT = "changeEvents";
public AtomixConsistentMap(CopycatClient client, Resource.Options options) {
super(client, options);
......@@ -68,14 +68,8 @@ public class AtomixConsistentMap extends Resource<AtomixConsistentMap, Resource.
});
}
private void handleEvent(MapEvent<String, byte[]> event) {
mapEventListeners.forEach(listener -> listener.event(event));
}
@Override
public AtomixConsistentMap with(Consistency consistency) {
super.with(consistency);
return this;
private void handleEvent(List<MapEvent<String, byte[]>> events) {
events.forEach(event -> mapEventListeners.forEach(listener -> listener.event(event)));
}
@Override
......
......@@ -18,7 +18,9 @@ package org.onosproject.store.primitives.resources.impl;
import io.atomix.catalyst.buffer.BufferInput;
import io.atomix.catalyst.buffer.BufferOutput;
import io.atomix.catalyst.serializer.CatalystSerializable;
import io.atomix.catalyst.serializer.SerializableTypeResolver;
import io.atomix.catalyst.serializer.Serializer;
import io.atomix.catalyst.serializer.SerializerRegistry;
import io.atomix.catalyst.util.Assert;
import io.atomix.copycat.client.Command;
import io.atomix.copycat.client.Query;
......@@ -514,4 +516,28 @@ public final class AtomixConsistentMapCommands {
.toString();
}
}
/**
* Map command type resolver.
*/
public static class TypeResolver implements SerializableTypeResolver {
@Override
public void resolve(SerializerRegistry registry) {
registry.register(ContainsKey.class, -761);
registry.register(ContainsValue.class, -762);
registry.register(Get.class, -763);
registry.register(EntrySet.class, -764);
registry.register(Values.class, -765);
registry.register(KeySet.class, -766);
registry.register(Clear.class, -767);
registry.register(IsEmpty.class, -768);
registry.register(Size.class, -769);
registry.register(Listen.class, -770);
registry.register(Unlisten.class, -771);
registry.register(TransactionPrepare.class, -772);
registry.register(TransactionCommit.class, -773);
registry.register(TransactionRollback.class, -774);
registry.register(UpdateAndGet.class, -775);
}
}
}
......
......@@ -17,7 +17,6 @@ package org.onosproject.store.primitives.resources.impl;
import io.atomix.catalyst.util.Listener;
import io.atomix.copycat.client.CopycatClient;
import io.atomix.resource.Consistency;
import io.atomix.resource.Resource;
import io.atomix.resource.ResourceTypeInfo;
......@@ -26,6 +25,7 @@ import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;
import org.onlab.util.SharedExecutors;
import org.onosproject.cluster.Leadership;
import org.onosproject.cluster.NodeId;
import org.onosproject.event.Change;
......@@ -37,8 +37,8 @@ import com.google.common.collect.Sets;
* Distributed resource providing the {@link AsyncLeaderElector} primitive.
*/
@ResourceTypeInfo(id = -152, stateMachine = AtomixLeaderElectorState.class)
public class AtomixLeaderElector
extends Resource<AtomixLeaderElector, Resource.Options> implements AsyncLeaderElector {
public class AtomixLeaderElector extends Resource<AtomixLeaderElector, Resource.Options>
implements AsyncLeaderElector {
private final Set<Consumer<Change<Leadership>>> leadershipChangeListeners =
Sets.newConcurrentHashSet();
......@@ -62,13 +62,8 @@ public class AtomixLeaderElector
}
private void handleEvent(Change<Leadership> change) {
leadershipChangeListeners.forEach(l -> l.accept(change));
}
@Override
public AtomixLeaderElector with(Consistency consistency) {
super.with(consistency);
return this;
SharedExecutors.getSingleThreadExecutor().execute(() ->
leadershipChangeListeners.forEach(l -> l.accept(change)));
}
@Override
......
......@@ -21,13 +21,16 @@ import java.util.Set;
import org.onosproject.cluster.Leadership;
import org.onosproject.cluster.NodeId;
import com.google.common.base.MoreObjects;
import com.google.common.base.Strings;
import io.atomix.catalyst.buffer.BufferInput;
import io.atomix.catalyst.buffer.BufferOutput;
import io.atomix.catalyst.serializer.CatalystSerializable;
import io.atomix.catalyst.serializer.SerializableTypeResolver;
import io.atomix.catalyst.serializer.Serializer;
import io.atomix.catalyst.serializer.SerializerRegistry;
import io.atomix.catalyst.util.Assert;
import io.atomix.copycat.client.Command;
import io.atomix.copycat.client.Query;
......@@ -232,6 +235,18 @@ public final class AtomixLeaderElectorCommands {
.add("nodeId", nodeId)
.toString();
}
@Override
public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
buffer.writeString(topic);
buffer.writeString(nodeId.toString());
}
@Override
public void readObject(BufferInput<?> buffer, Serializer serializer) {
topic = buffer.readString();
nodeId = new NodeId(buffer.readString());
}
}
/**
......@@ -263,6 +278,16 @@ public final class AtomixLeaderElectorCommands {
.add("topic", topic)
.toString();
}
@Override
public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
buffer.writeString(topic);
}
@Override
public void readObject(BufferInput<?> buffer, Serializer serializer) {
topic = buffer.readString();
}
}
/**
......@@ -306,5 +331,34 @@ public final class AtomixLeaderElectorCommands {
.add("nodeId", nodeId)
.toString();
}
@Override
public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
buffer.writeString(topic);
buffer.writeString(nodeId.toString());
}
@Override
public void readObject(BufferInput<?> buffer, Serializer serializer) {
topic = buffer.readString();
nodeId = new NodeId(buffer.readString());
}
}
/**
* Map command type resolver.
*/
public static class TypeResolver implements SerializableTypeResolver {
@Override
public void resolve(SerializerRegistry registry) {
registry.register(Run.class, -861);
registry.register(Withdraw.class, -862);
registry.register(Anoint.class, -863);
registry.register(GetAllLeaderships.class, -864);
registry.register(GetElectedTopics.class, -865);
registry.register(GetLeadership.class, -866);
registry.register(Listen.class, -867);
registry.register(Unlisten.class, -868);
}
}
}
......
......@@ -41,6 +41,14 @@ import org.onosproject.cluster.Leader;
import org.onosproject.cluster.Leadership;
import org.onosproject.cluster.NodeId;
import org.onosproject.event.Change;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Anoint;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetAllLeaderships;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetElectedTopics;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetLeadership;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Listen;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Run;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Unlisten;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Withdraw;
import org.onosproject.store.serializers.KryoNamespaces;
import org.onosproject.store.service.Serializer;
import org.slf4j.Logger;
......@@ -59,7 +67,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
private final Logger log = getLogger(getClass());
private Map<String, AtomicLong> termCounters = new HashMap<>();
private Map<String, ElectionState> elections = new HashMap<>();
private final Map<Long, Commit<? extends AtomixLeaderElectorCommands.Listen>> listeners = new LinkedHashMap<>();
private final Map<Long, Commit<? extends Listen>> listeners = new LinkedHashMap<>();
private final Serializer serializer = Serializer.using(Arrays.asList(KryoNamespaces.API),
ElectionState.class,
Registration.class);
......@@ -67,16 +75,16 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
@Override
protected void configure(StateMachineExecutor executor) {
// Notification
executor.register(AtomixLeaderElectorCommands.Listen.class, this::listen);
executor.register(AtomixLeaderElectorCommands.Unlisten.class, this::unlisten);
executor.register(Listen.class, this::listen);
executor.register(Unlisten.class, this::unlisten);
// Commands
executor.register(AtomixLeaderElectorCommands.Run.class, this::run);
executor.register(AtomixLeaderElectorCommands.Withdraw.class, this::withdraw);
executor.register(AtomixLeaderElectorCommands.Anoint.class, this::anoint);
executor.register(Run.class, this::run);
executor.register(Withdraw.class, this::withdraw);
executor.register(Anoint.class, this::anoint);
// Queries
executor.register(AtomixLeaderElectorCommands.GetLeadership.class, this::leadership);
executor.register(AtomixLeaderElectorCommands.GetAllLeaderships.class, this::allLeaderships);
executor.register(AtomixLeaderElectorCommands.GetElectedTopics.class, this::electedTopics);
executor.register(GetLeadership.class, this::leadership);
executor.register(GetAllLeaderships.class, this::allLeaderships);
executor.register(GetElectedTopics.class, this::electedTopics);
}
private void notifyLeadershipChange(Leadership previousLeadership, Leadership newLeadership) {
......@@ -96,7 +104,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
*
* @param commit listen commit
*/
public void listen(Commit<? extends AtomixLeaderElectorCommands.Listen> commit) {
public void listen(Commit<? extends Listen> commit) {
if (listeners.putIfAbsent(commit.session().id(), commit) != null) {
commit.close();
}
......@@ -107,9 +115,9 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
*
* @param commit unlisten commit
*/
public void unlisten(Commit<? extends AtomixLeaderElectorCommands.Unlisten> commit) {
public void unlisten(Commit<? extends Unlisten> commit) {
try {
Commit<? extends AtomixLeaderElectorCommands.Listen> listener = listeners.remove(commit.session().id());
Commit<? extends Listen> listener = listeners.remove(commit.session().id());
if (listener != null) {
listener.close();
}
......@@ -123,7 +131,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
* @param commit commit entry
* @return topic leader. If no previous leader existed this is the node that just entered the race.
*/
public Leadership run(Commit<? extends AtomixLeaderElectorCommands.Run> commit) {
public Leadership run(Commit<? extends Run> commit) {
try {
String topic = commit.operation().topic();
Leadership oldLeadership = leadership(topic);
......@@ -154,7 +162,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
* Applies an {@link AtomixLeaderElectorCommands.Withdraw} commit.
* @param commit withdraw commit
*/
public void withdraw(Commit<? extends AtomixLeaderElectorCommands.Withdraw> commit) {
public void withdraw(Commit<? extends Withdraw> commit) {
try {
String topic = commit.operation().topic();
Leadership oldLeadership = leadership(topic);
......@@ -174,7 +182,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
* @param commit anoint commit
* @return {@code true} if changes were made and the transfer occurred; {@code false} if it did not.
*/
public boolean anoint(Commit<? extends AtomixLeaderElectorCommands.Anoint> commit) {
public boolean anoint(Commit<? extends Anoint> commit) {
try {
String topic = commit.operation().topic();
Leadership oldLeadership = leadership(topic);
......@@ -197,7 +205,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
* @param commit GetLeadership commit
* @return leader
*/
public Leadership leadership(Commit<? extends AtomixLeaderElectorCommands.GetLeadership> commit) {
public Leadership leadership(Commit<? extends GetLeadership> commit) {
String topic = commit.operation().topic();
try {
return leadership(topic);
......@@ -211,7 +219,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
* @param commit commit entry
* @return set of topics for which the node is the leader
*/
public Set<String> electedTopics(Commit<? extends AtomixLeaderElectorCommands.GetElectedTopics> commit) {
public Set<String> electedTopics(Commit<? extends GetElectedTopics> commit) {
try {
NodeId nodeId = commit.operation().nodeId();
return Maps.filterEntries(elections, e -> {
......@@ -228,8 +236,7 @@ public class AtomixLeaderElectorState extends ResourceStateMachine
* @param commit GetAllLeaderships commit
* @return topic to leader mapping
*/
public Map<String, Leadership> allLeaderships(
Commit<? extends AtomixLeaderElectorCommands.GetAllLeaderships> commit) {
public Map<String, Leadership> allLeaderships(Commit<? extends GetAllLeaderships> commit) {
try {
return Maps.transformEntries(elections, (k, v) -> leadership(k));
} finally {
......