Madan Jampani
Committed by Gerrit Code Review

State machine implementations for various distributed primitives based on latest Copycat APIs

Change-Id: I622cc196aa1cdf072a5a0b100a5ffaaf71b07900
Showing 21 changed files with 1692 additions and 0 deletions
/*
* Copyright 2016 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.primitives.impl;
import io.atomix.catalyst.serializer.CatalystSerializable;
import io.atomix.catalyst.serializer.Serializer;
import io.atomix.catalyst.serializer.TypeSerializerFactory;
import io.atomix.copycat.client.Query;
import io.atomix.manager.state.GetResource;
import io.atomix.manager.state.GetResourceKeys;
import io.atomix.resource.ResourceQuery;
import java.io.IOException;
import java.net.URL;
import java.util.Arrays;
import java.util.Enumeration;
import java.util.Scanner;
import org.onlab.util.Match;
import org.onosproject.cluster.NodeId;
import org.onosproject.event.Change;
import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands;
import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapState;
import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands;
import org.onosproject.store.primitives.resources.impl.CommitResult;
import org.onosproject.store.primitives.resources.impl.MapEntryUpdateResult;
import org.onosproject.store.primitives.resources.impl.MapUpdate;
import org.onosproject.store.primitives.resources.impl.PrepareResult;
import org.onosproject.store.primitives.resources.impl.RollbackResult;
import org.onosproject.store.primitives.resources.impl.TransactionId;
import org.onosproject.store.primitives.resources.impl.TransactionalMapUpdate;
import org.onosproject.store.serializers.KryoNamespaces;
import org.onosproject.store.service.MapEvent;
import org.onosproject.store.service.Versioned;
import com.google.common.base.Throwables;
import com.google.common.collect.Maps;
/**
* Serializer utility for Atomix Catalyst.
*/
public final class CatalystSerializers {
private CatalystSerializers() {
}
public static Serializer getSerializer() {
Serializer serializer = new Serializer();
TypeSerializerFactory factory =
new DefaultCatalystTypeSerializerFactory(
org.onosproject.store.service.Serializer.using(Arrays.asList((KryoNamespaces.API)),
MapEntryUpdateResult.class,
MapEntryUpdateResult.Status.class,
MapUpdate.class,
MapUpdate.Type.class,
TransactionalMapUpdate.class,
TransactionId.class,
PrepareResult.class,
CommitResult.class,
RollbackResult.class,
AtomixConsistentMapCommands.Get.class,
AtomixConsistentMapCommands.ContainsKey.class,
AtomixConsistentMapCommands.ContainsValue.class,
AtomixConsistentMapCommands.Size.class,
AtomixConsistentMapCommands.IsEmpty.class,
AtomixConsistentMapCommands.KeySet.class,
AtomixConsistentMapCommands.EntrySet.class,
AtomixConsistentMapCommands.Values.class,
AtomixConsistentMapCommands.UpdateAndGet.class,
AtomixConsistentMapCommands.TransactionPrepare.class,
AtomixConsistentMapCommands.TransactionCommit.class,
AtomixConsistentMapCommands.TransactionRollback.class,
AtomixLeaderElectorCommands.GetLeadership.class,
AtomixLeaderElectorCommands.GetAllLeaderships.class,
AtomixLeaderElectorCommands.GetElectedTopics.class,
AtomixLeaderElectorCommands.Run.class,
AtomixLeaderElectorCommands.Withdraw.class,
AtomixLeaderElectorCommands.Anoint.class,
GetResource.class,
GetResourceKeys.class,
ResourceQuery.class,
Query.ConsistencyLevel.class));
// ONOS classes
serializer.register(Change.class, factory);
serializer.register(NodeId.class, factory);
serializer.register(Match.class, factory);
serializer.register(MapEntryUpdateResult.class, factory);
serializer.register(MapEntryUpdateResult.Status.class, factory);
serializer.register(TransactionalMapUpdate.class, factory);
serializer.register(PrepareResult.class, factory);
serializer.register(CommitResult.class, factory);
serializer.register(RollbackResult.class, factory);
serializer.register(TransactionId.class, factory);
serializer.register(MapUpdate.class, factory);
serializer.register(Versioned.class, factory);
serializer.register(MapEvent.class, factory);
serializer.register(Maps.immutableEntry("a", "b").getClass(), factory);
serializer.register(AtomixConsistentMapState.class, factory);
serializer.register(ResourceQuery.class, factory);
serializer.register(GetResource.class, factory);
serializer.register(GetResourceKeys.class, factory);
// ConsistentMap
serializer.register(AtomixConsistentMapCommands.UpdateAndGet.class, factory);
serializer.register(AtomixConsistentMapCommands.Clear.class);
serializer.register(AtomixConsistentMapCommands.Listen.class);
serializer.register(AtomixConsistentMapCommands.Unlisten.class);
serializer.register(AtomixConsistentMapCommands.Get.class);
serializer.register(AtomixConsistentMapCommands.ContainsKey.class);
serializer.register(AtomixConsistentMapCommands.ContainsValue.class);
serializer.register(AtomixConsistentMapCommands.EntrySet.class);
serializer.register(AtomixConsistentMapCommands.IsEmpty.class);
serializer.register(AtomixConsistentMapCommands.KeySet.class);
serializer.register(AtomixConsistentMapCommands.Size.class);
serializer.register(AtomixConsistentMapCommands.Values.class);
serializer.register(AtomixConsistentMapCommands.TransactionPrepare.class);
serializer.register(AtomixConsistentMapCommands.TransactionCommit.class);
serializer.register(AtomixConsistentMapCommands.TransactionRollback.class);
// LeaderElector
serializer.register(AtomixLeaderElectorCommands.Run.class, factory);
serializer.register(AtomixLeaderElectorCommands.Withdraw.class, factory);
serializer.register(AtomixLeaderElectorCommands.Anoint.class, factory);
serializer.register(AtomixLeaderElectorCommands.GetElectedTopics.class, factory);
serializer.register(AtomixLeaderElectorCommands.GetElectedTopics.class, factory);
serializer.register(AtomixLeaderElectorCommands.GetLeadership.class, factory);
serializer.register(AtomixLeaderElectorCommands.GetAllLeaderships.class, factory);
serializer.register(AtomixLeaderElectorCommands.Listen.class);
serializer.register(AtomixLeaderElectorCommands.Unlisten.class);
// Atomix types
try {
ClassLoader cl = CatalystSerializable.class.getClassLoader();
Enumeration<URL> urls = cl.getResources(
String.format("META-INF/services/%s", CatalystSerializable.class.getName()));
while (urls.hasMoreElements()) {
URL url = urls.nextElement();
try (Scanner scanner = new Scanner(url.openStream(), "UTF-8")) {
scanner.useDelimiter("\n").forEachRemaining(line -> {
if (!line.trim().startsWith("#")) {
line = line.trim();
if (line.length() > 0) {
try {
serializer.register(cl.loadClass(line));
} catch (ClassNotFoundException e) {
Throwables.propagate(e);
}
}
}
});
}
}
} catch (IOException e) {
Throwables.propagate(e);
}
return serializer;
}
}
/*
* Copyright 2016 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.primitives.resources.impl;
import io.atomix.variables.DistributedLong;
import java.util.concurrent.CompletableFuture;
import org.onosproject.store.service.AsyncAtomicCounter;
/**
* {@code AsyncAtomicCounter} implementation backed by Atomix
* {@link DistributedLong}.
*/
public class AtomixCounter implements AsyncAtomicCounter {
private final String name;
private final DistributedLong distLong;
public AtomixCounter(String name, DistributedLong distLong) {
this.name = name;
this.distLong = distLong;
}
@Override
public String name() {
return name;
}
@Override
public CompletableFuture<Long> incrementAndGet() {
return distLong.incrementAndGet();
}
@Override
public CompletableFuture<Long> getAndIncrement() {
return distLong.getAndIncrement();
}
@Override
public CompletableFuture<Long> getAndAdd(long delta) {
return distLong.getAndAdd(delta);
}
@Override
public CompletableFuture<Long> addAndGet(long delta) {
return distLong.addAndGet(delta);
}
@Override
public CompletableFuture<Long> get() {
return distLong.get();
}
@Override
public CompletableFuture<Void> set(long value) {
return distLong.set(value);
}
@Override
public CompletableFuture<Boolean> compareAndSet(long expectedValue,
long updateValue) {
return distLong.compareAndSet(expectedValue, updateValue);
}
}
\ No newline at end of file
/*
* Copyright 2016 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.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.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;
import org.onosproject.cluster.Leadership;
import org.onosproject.cluster.NodeId;
import org.onosproject.event.Change;
import org.onosproject.store.service.AsyncLeaderElector;
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 {
private final Set<Consumer<Change<Leadership>>> leadershipChangeListeners =
Sets.newConcurrentHashSet();
private Listener<Change<Leadership>> listener;
public AtomixLeaderElector(CopycatClient client, Resource.Options options) {
super(client, options);
}
@Override
public String name() {
return null;
}
@Override
public CompletableFuture<AtomixLeaderElector> open() {
return super.open().thenApply(result -> {
client.session().onEvent("change", this::handleEvent);
return result;
});
}
private void handleEvent(Change<Leadership> change) {
leadershipChangeListeners.forEach(l -> l.accept(change));
}
@Override
public AtomixLeaderElector with(Consistency consistency) {
super.with(consistency);
return this;
}
@Override
public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
return submit(new AtomixLeaderElectorCommands.Run(topic, nodeId));
}
@Override
public CompletableFuture<Void> withdraw(String topic) {
return submit(new AtomixLeaderElectorCommands.Withdraw(topic));
}
@Override
public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
return submit(new AtomixLeaderElectorCommands.Anoint(topic, nodeId));
}
@Override
public CompletableFuture<Leadership> getLeadership(String topic) {
return submit(new AtomixLeaderElectorCommands.GetLeadership(topic));
}
@Override
public CompletableFuture<Map<String, Leadership>> getLeaderships() {
return submit(new AtomixLeaderElectorCommands.GetAllLeaderships());
}
public CompletableFuture<Set<String>> getElectedTopics(NodeId nodeId) {
return submit(new AtomixLeaderElectorCommands.GetElectedTopics(nodeId));
}
/**
* Leadership change listener context.
*/
private final class LeadershipChangeListener implements Listener<Change<Leadership>> {
private final Consumer<Change<Leadership>> listener;
private LeadershipChangeListener(Consumer<Change<Leadership>> listener) {
this.listener = listener;
}
@Override
public void accept(Change<Leadership> change) {
listener.accept(change);
}
@Override
public void close() {
synchronized (AtomixLeaderElector.this) {
submit(new AtomixLeaderElectorCommands.Unlisten());
}
}
}
@Override
public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
leadershipChangeListeners.add(consumer);
return setupListener();
}
@Override
public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
leadershipChangeListeners.remove(consumer);
return teardownListener();
}
private CompletableFuture<Void> setupListener() {
if (listener == null && !leadershipChangeListeners.isEmpty()) {
Consumer<Change<Leadership>> changeConsumer = change -> {
leadershipChangeListeners.forEach(consumer -> consumer.accept(change));
};
return submit(new AtomixLeaderElectorCommands.Listen())
.thenAccept(v -> listener = new LeadershipChangeListener(changeConsumer));
}
return CompletableFuture.completedFuture(null);
}
private CompletableFuture<Void> teardownListener() {
if (listener != null && leadershipChangeListeners.isEmpty()) {
listener.close();
listener = null;
return submit(new AtomixLeaderElectorCommands.Unlisten());
}
return CompletableFuture.completedFuture(null);
}
}
/*
* Copyright 2016 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.primitives.resources.impl;
import java.util.Map;
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.Serializer;
import io.atomix.catalyst.util.Assert;
import io.atomix.copycat.client.Command;
import io.atomix.copycat.client.Query;
/**
* {@link AtomixLeaderElector} resource state machine operations.
*/
public final class AtomixLeaderElectorCommands {
private AtomixLeaderElectorCommands() {
}
/**
* Abstract election query.
*/
@SuppressWarnings("serial")
public abstract static class ElectionQuery<V> implements Query<V>, CatalystSerializable {
@Override
public ConsistencyLevel consistency() {
return ConsistencyLevel.BOUNDED_LINEARIZABLE;
}
@Override
public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
}
@Override
public void readObject(BufferInput<?> buffer, Serializer serializer) {
}
}
/**
* Abstract election topic query.
*/
@SuppressWarnings("serial")
public abstract static class TopicQuery<V> extends ElectionQuery<V> implements CatalystSerializable {
String topic;
public TopicQuery() {
}
public TopicQuery(String topic) {
this.topic = Assert.notNull(topic, "topic");
}
/**
* Returns the topic.
* @return topic
*/
public String topic() {
return topic;
}
@Override
public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
serializer.writeObject(topic, buffer);
}
@Override
public void readObject(BufferInput<?> buffer, Serializer serializer) {
topic = serializer.readObject(buffer);
}
}
/**
* Abstract election command.
*/
@SuppressWarnings("serial")
public abstract static class ElectionCommand<V> implements Command<V>, CatalystSerializable {
@Override
public ConsistencyLevel consistency() {
return ConsistencyLevel.LINEARIZABLE;
}
@Override
public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
}
@Override
public void readObject(BufferInput<?> buffer, Serializer serializer) {
}
}
/**
* Listen command.
*/
@SuppressWarnings("serial")
public static class Listen extends ElectionCommand<Void> {
}
/**
* Unlisten command.
*/
@SuppressWarnings("serial")
public static class Unlisten extends ElectionCommand<Void> {
@Override
public CompactionMode compaction() {
return CompactionMode.QUORUM;
}
}
/**
* GetLeader query.
*/
@SuppressWarnings("serial")
public static class GetLeadership extends TopicQuery<Leadership> {
public GetLeadership() {
}
public GetLeadership(String topic) {
super(topic);
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("topic", topic)
.toString();
}
}
/**
* GetAllLeaders query.
*/
@SuppressWarnings("serial")
public static class GetAllLeaderships extends ElectionQuery<Map<String, Leadership>> {
}
/**
* GetElectedTopics query.
*/
@SuppressWarnings("serial")
public static class GetElectedTopics extends ElectionQuery<Set<String>> {
private NodeId nodeId;
public GetElectedTopics() {
}
public GetElectedTopics(NodeId nodeId) {
this.nodeId = Assert.argNot(nodeId, nodeId == null, "nodeId cannot be null");
}
/**
* Returns the nodeId to check.
*
* @return The nodeId to check.
*/
public NodeId nodeId() {
return nodeId;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("nodeId", nodeId)
.toString();
}
}
/**
* Enter and run for leadership.
*/
@SuppressWarnings("serial")
public static class Run extends ElectionCommand<Leadership> {
private String topic;
private NodeId nodeId;
public Run() {
}
public Run(String topic, NodeId nodeId) {
this.topic = Assert.argNot(topic, Strings.isNullOrEmpty(topic), "topic cannot be null or empty");
this.nodeId = Assert.argNot(nodeId, nodeId == null, "nodeId cannot be null");
}
/**
* Returns the topic.
*
* @return topic
*/
public String topic() {
return topic;
}
/**
* Returns the nodeId.
*
* @return the nodeId
*/
public NodeId nodeId() {
return nodeId;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("topic", topic)
.add("nodeId", nodeId)
.toString();
}
}
/**
* Withdraw from a leadership contest.
*/
@SuppressWarnings("serial")
public static class Withdraw extends ElectionCommand<Void> {
private String topic;
public Withdraw() {
}
public Withdraw(String topic) {
this.topic = Assert.argNot(topic, Strings.isNullOrEmpty(topic), "topic cannot be null or empty");
}
/**
* Returns the topic.
*
* @return The topic
*/
public String topic() {
return topic;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("topic", topic)
.toString();
}
}
/**
* Command for administratively anointing a node as leader.
*/
@SuppressWarnings("serial")
public static class Anoint extends ElectionCommand<Boolean> {
private String topic;
private NodeId nodeId;
public Anoint() {
}
public Anoint(String topic, NodeId nodeId) {
this.topic = topic;
this.nodeId = nodeId;
}
/**
* Returns the topic.
*
* @return The topic
*/
public String topic() {
return topic;
}
/**
* Returns the nodeId to make leader.
*
* @return The nodeId
*/
public NodeId nodeId() {
return nodeId;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(getClass())
.add("topic", topic)
.add("nodeId", nodeId)
.toString();
}
}
}
/*
* Copyright 2016 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.primitives.resources.impl;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import org.onlab.util.Tools;
import org.onosproject.store.service.AsyncAtomicValue;
import org.onosproject.store.service.AtomicValueEvent;
import org.onosproject.store.service.AtomicValueEventListener;
import org.onosproject.store.service.MapEventListener;
import org.onosproject.store.service.Versioned;
import com.google.common.collect.Sets;
/**
* Implementation of {@link AsyncAtomicValue} backed by {@link AtomixConsistentMap}.
*/
public class AtomixValue implements AsyncAtomicValue<String> {
private final String name;
private final AtomixConsistentMap atomixMap;
private MapEventListener<String, byte[]> mapEventListener;
private final Set<AtomicValueEventListener<String>> listeners = Sets.newIdentityHashSet();
AtomixValue(String name, AtomixConsistentMap atomixMap) {
this.name = name;
this.atomixMap = atomixMap;
}
@Override
public CompletableFuture<Boolean> compareAndSet(String expect, String update) {
return atomixMap.replace(name, Tools.getBytesUtf8(expect), Tools.getBytesUtf8(update));
}
@Override
public CompletableFuture<String> get() {
return atomixMap.get(name)
.thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
}
@Override
public CompletableFuture<String> getAndSet(String value) {
return atomixMap.put(name, Tools.getBytesUtf8(value))
.thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
}
@Override
public CompletableFuture<Void> set(String value) {
return getAndSet(value).thenApply(v -> null);
}
@Override
public CompletableFuture<Void> addListener(AtomicValueEventListener<String> listener) {
// TODO: synchronization
if (mapEventListener == null) {
mapEventListener = event -> {
Versioned<byte[]> newValue = event.newValue();
Versioned<byte[]> oldValue = event.oldValue();
if (Objects.equals(event.key(), name)) {
listener.event(new AtomicValueEvent<>(name,
newValue == null ? null : Tools.toStringUtf8(newValue.value()),
oldValue == null ? null : Tools.toStringUtf8(oldValue.value())));
}
};
return atomixMap.addListener(mapEventListener).whenComplete((r, e) -> {
if (e == null) {
listeners.add(listener);
} else {
mapEventListener = null;
}
});
} else {
listeners.add(listener);
return CompletableFuture.completedFuture(null);
}
}
@Override
public CompletableFuture<Void> removeListener(AtomicValueEventListener<String> listener) {
// TODO: synchronization
listeners.remove(listener);
if (listeners.isEmpty()) {
return atomixMap.removeListener(mapEventListener);
} else {
return CompletableFuture.completedFuture(null);
}
}
@Override
public String name() {
return null;
}
}
\ No newline at end of file
/*
* Copyright 2016 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.primitives.resources.impl;
/**
* Response enum for two phase commit operation.
*/
public enum CommitResult {
/**
* Signifies a successful commit execution.
*/
OK,
/**
* Signifies a failure due to unrecognized transaction identifier.
*/
UNKNOWN_TRANSACTION_ID,
}
/*
* 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.primitives.resources.impl;
import java.util.function.Function;
import org.onosproject.store.service.MapEvent;
import org.onosproject.store.service.Versioned;
import com.google.common.base.MoreObjects;
/**
* Result of a map entry update operation.
* <p>
* Both old and new values are accessible along with a flag that indicates if the
* the value was updated. If flag is false, oldValue and newValue both
* point to the same unmodified value.
* @param <V> result type
*/
public class MapEntryUpdateResult<K, V> {
public enum Status {
/**
* Indicates a successful update.
*/
OK,
/**
* Indicates a noop i.e. existing and new value are both null.
*/
NOOP,
/**
* Indicates a failed update due to a write lock.
*/
WRITE_LOCK,
/**
* Indicates a failed update due to a precondition check failure.
*/
PRECONDITION_FAILED
}
private final String mapName;
private Status status;
private final K key;
private final Versioned<V> oldValue;
private final Versioned<V> newValue;
public MapEntryUpdateResult(Status status, String mapName, K key, Versioned<V> oldValue, Versioned<V> newValue) {
this.status = status;
this.mapName = mapName;
this.key = key;
this.oldValue = oldValue;
this.newValue = newValue;
}
/**
* Returns {@code true} if the update was successful.
* @return {@code true} if yes, {@code false} otherwise
*/
public boolean updated() {
return status == Status.OK;
}
/**
* Returns the map name.
* @return map name
*/
public String mapName() {
return mapName;
}
/**
* Returns the update status.
* @return update status
*/
public Status status() {
return status;
}
/**
* Returns the map key.
* @return key
*/
public K key() {
return key;
}
/**
* Returns the old value.
* @return the previous value associated with key if updated was successful, otherwise current value
*/
public Versioned<V> oldValue() {
return oldValue;
}
/**
* Returns the new value after update.
* @return if updated was unsuccessful, this is same as old value
*/
public Versioned<V> newValue() {
return newValue;
}
/**
* Maps to another instance with different key and value types.
* @param keyTransform transformer to use for transcoding keys
* @param valueMapper mapper to use for transcoding values
* @return new instance
*/
public <K1, V1> MapEntryUpdateResult<K1, V1> map(Function<K, K1> keyTransform, Function<V, V1> valueMapper) {
return new MapEntryUpdateResult<>(status,
mapName,
keyTransform.apply(key),
oldValue == null ? null : oldValue.map(valueMapper),
newValue == null ? null : newValue.map(valueMapper));
}
/**
* Return the map event that will be generated as a result of this update.
* @return map event. if update was unsuccessful, this returns {@code null}
*/
public MapEvent<K, V> toMapEvent() {
if (!updated()) {
return null;
} else {
return new MapEvent<>(mapName(), key(), newValue, oldValue);
}
}
@Override
public String toString() {
return MoreObjects.toStringHelper(MapEntryUpdateResult.class)
.add("mapName", mapName)
.add("status", status)
.add("key", key)
.add("oldValue", oldValue)
.add("newValue", newValue)
.toString();
}
}
/*
* 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.primitives.resources.impl;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import com.google.common.base.MoreObjects;
/**
* Map update operation.
*
* @param <K> map key type
* @param <V> map value type
*
*/
public final class MapUpdate<K, V> {
/**
* Type of database update operation.
*/
public enum Type {
/**
* Insert/Update entry without any checks.
*/
PUT,
/**
* Insert an entry iff there is no existing entry for that key.
*/
PUT_IF_ABSENT,
/**
* Update entry if the current version matches specified version.
*/
PUT_IF_VERSION_MATCH,
/**
* Update entry if the current value matches specified value.
*/
PUT_IF_VALUE_MATCH,
/**
* Remove entry without any checks.
*/
REMOVE,
/**
* Remove entry if the current version matches specified version.
*/
REMOVE_IF_VERSION_MATCH,
/**
* Remove entry if the current value matches specified value.
*/
REMOVE_IF_VALUE_MATCH,
}
private Type type;
private K key;
private V value;
private V currentValue;
private long currentVersion = -1;
/**
* Returns the type of update operation.
* @return type of update.
*/
public Type type() {
return type;
}
/**
* Returns the item key being updated.
* @return item key
*/
public K key() {
return key;
}
/**
* Returns the new value.
* @return item's target value.
*/
public V value() {
return value;
}
/**
* Returns the expected current value for the key.
* @return current value in database.
*/
public V currentValue() {
return currentValue;
}
/**
* Returns the expected current version in the database for the key.
* @return expected version.
*/
public long currentVersion() {
return currentVersion;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("type", type)
.add("key", key)
.add("value", value)
.add("currentValue", currentValue)
.add("currentVersion", currentVersion)
.toString();
}
/**
* Creates a new builder instance.
*
* @param <K> key type
* @param <V> value type
* @return builder.
*/
public static <K, V> Builder<K, V> newBuilder() {
return new Builder<>();
}
/**
* MapUpdate builder.
*
* @param <K> key type
* @param <V> value type
*/
public static final class Builder<K, V> {
private MapUpdate<K, V> update = new MapUpdate<>();
public MapUpdate<K, V> build() {
validateInputs();
return update;
}
public Builder<K, V> withType(Type type) {
update.type = checkNotNull(type, "type cannot be null");
return this;
}
public Builder<K, V> withKey(K key) {
update.key = checkNotNull(key, "key cannot be null");
return this;
}
public Builder<K, V> withCurrentValue(V value) {
update.currentValue = checkNotNull(value, "currentValue cannot be null");
return this;
}
public Builder<K, V> withValue(V value) {
update.value = checkNotNull(value, "value cannot be null");
return this;
}
public Builder<K, V> withCurrentVersion(long version) {
checkArgument(version >= 0, "version cannot be negative");
update.currentVersion = version;
return this;
}
private void validateInputs() {
checkNotNull(update.type, "type must be specified");
checkNotNull(update.key, "key must be specified");
switch (update.type) {
case PUT:
case PUT_IF_ABSENT:
checkNotNull(update.value, "value must be specified.");
break;
case PUT_IF_VERSION_MATCH:
checkNotNull(update.value, "value must be specified.");
checkState(update.currentVersion >= 0, "current version must be specified");
break;
case PUT_IF_VALUE_MATCH:
checkNotNull(update.value, "value must be specified.");
checkNotNull(update.currentValue, "currentValue must be specified.");
break;
case REMOVE:
break;
case REMOVE_IF_VERSION_MATCH:
checkState(update.currentVersion >= 0, "current version must be specified");
break;
case REMOVE_IF_VALUE_MATCH:
checkNotNull(update.currentValue, "currentValue must be specified.");
break;
default:
throw new IllegalStateException("Unknown operation type");
}
}
}
}
/*
* Copyright 2016 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.primitives.resources.impl;
/**
* Response enum for two phase commit prepare operation.
*/
public enum PrepareResult {
/**
* Signifies a successful execution of the prepare operation.
*/
OK,
/**
* Signifies a failure to another transaction locking the underlying state.
*/
CONCURRENT_TRANSACTION,
/**
* Signifies a optimistic lock failure. This can happen if underlying state has changed since it was last read.
*/
OPTIMISTIC_LOCK_FAILURE,
}
\ No newline at end of file
/*
* Copyright 2016 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.primitives.resources.impl;
/**
* Response enum for two phase commit rollback operation.
*/
public enum RollbackResult {
/**
* Signifies a successful rollback execution.
*/
OK,
/**
* Signifies a failure due to unrecognized transaction identifier.
*/
UNKNOWN_TRANSACTION_ID,
}
/*
* Copyright 2016 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.primitives.resources.impl;
import com.google.common.base.Objects;
/**
* Transaction identifier.
*/
public final class TransactionId {
public static TransactionId from(String id) {
return new TransactionId(id);
}
private final String id;
private TransactionId(String id) {
this.id = id;
}
@Override
public String toString() {
return id;
}
@Override
public int hashCode() {
return id.hashCode();
}
@Override
public boolean equals(Object other) {
if (this == other) {
return true;
}
if (other instanceof TransactionId) {
TransactionId that = (TransactionId) other;
return Objects.equal(this.id, that.id);
}
return false;
}
}
/*
* Copyright 2016 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.primitives.resources.impl;
import java.util.Collection;
import java.util.Map;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Maps;
/**
* A batch updates to an {@code AsyncConsistentMap} be committed as a transaction.
*
* @param <K> key type
* @param <V> value type
*/
public class TransactionalMapUpdate<K, V> {
private final TransactionId transactionId;
private final Collection<MapUpdate<K, V>> updates;
private boolean indexPopulated = false;
private final Map<K, V> keyValueIndex = Maps.newHashMap();
public TransactionalMapUpdate(TransactionId transactionId, Collection<MapUpdate<K, V>> updates) {
this.transactionId = transactionId;
this.updates = ImmutableList.copyOf(updates);
populateIndex();
}
/**
* Returns the transaction identifier.
* @return transaction id
*/
public TransactionId transactionId() {
return transactionId;
}
/**
* Returns the collection of map updates.
* @return map updates
*/
public Collection<MapUpdate<K, V>> batch() {
return updates;
}
/**
* Returns the value that will be associated with the key after this transaction commits.
* @param key key
* @return value that will be associated with the value once this transaction commits
*/
public V valueForKey(K key) {
if (!indexPopulated) {
// We do not synchronize as we don't expect this called to be made from multiple threads.
populateIndex();
}
return keyValueIndex.get(key);
}
/**
* Populates the internal key -> value mapping.
*/
private synchronized void populateIndex() {
updates.forEach(mapUpdate -> {
if (mapUpdate.value() != null) {
keyValueIndex.put(mapUpdate.key(), mapUpdate.value());
}
});
indexPopulated = true;
}
}
/*
* Copyright 2016 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.
*/
/**
* State machine implementation classes for distributed primitives.
*/
package org.onosproject.store.primitives.resources.impl;
/*
* Copyright 2016 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.primitives.resources.impl;
import static org.junit.Assert.*;
import org.junit.Test;
import io.atomix.Atomix;
import io.atomix.resource.ResourceType;
import io.atomix.variables.DistributedLong;
/**
* Unit tests for {@link AtomixCounter}.
*/
public class AtomixLongTest extends AtomixTestBase {
@Override
protected ResourceType resourceType() {
return new ResourceType(DistributedLong.class);
}
@Test
public void testBasicOperations() throws Throwable {
basicOperationsTest(1);
clearTests();
basicOperationsTest(2);
clearTests();
basicOperationsTest(3);
clearTests();
}
protected void basicOperationsTest(int clusterSize) throws Throwable {
createCopycatServers(clusterSize);
Atomix atomix = createAtomixClient();
AtomixCounter along = new AtomixCounter("test-long", atomix.getLong("test-long").join());
assertEquals(0, along.get().join().longValue());
assertEquals(1, along.incrementAndGet().join().longValue());
along.set(100).join();
assertEquals(100, along.get().join().longValue());
assertEquals(100, along.getAndAdd(10).join().longValue());
assertEquals(110, along.get().join().longValue());
assertFalse(along.compareAndSet(109, 111).join());
assertTrue(along.compareAndSet(110, 111).join());
assertEquals(100, along.addAndGet(-11).join().longValue());
assertEquals(100, along.getAndIncrement().join().longValue());
assertEquals(101, along.get().join().longValue());
}
}
/*
* Copyright 2016 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.primitives.resources.impl;
import io.atomix.Atomix;
import io.atomix.AtomixClient;
import io.atomix.catalyst.serializer.Serializer;
import io.atomix.catalyst.transport.Address;
import io.atomix.catalyst.transport.LocalServerRegistry;
import io.atomix.catalyst.transport.LocalTransport;
import io.atomix.copycat.client.CopycatClient;
import io.atomix.copycat.server.CopycatServer;
import io.atomix.copycat.server.storage.Storage;
import io.atomix.copycat.server.storage.StorageLevel;
import io.atomix.manager.state.ResourceManagerState;
import io.atomix.resource.ResourceRegistry;
import io.atomix.resource.ResourceType;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.time.Duration;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import org.junit.After;
import org.junit.Before;
import org.onosproject.store.primitives.impl.CatalystSerializers;
import com.google.common.util.concurrent.Uninterruptibles;
/**
* Base class for various Atomix* tests.
*/
public abstract class AtomixTestBase {
private static final File TEST_DIR = new File("target/test-logs");
protected LocalServerRegistry registry;
protected int port;
protected List<Address> members;
protected List<CopycatClient> copycatClients = new ArrayList<>();
protected List<CopycatServer> copycatServers = new ArrayList<>();
protected List<Atomix> atomixClients = new ArrayList<>();
protected List<CopycatServer> atomixServers = new ArrayList<>();
protected Serializer serializer = CatalystSerializers.getSerializer();
/**
* Creates a new resource state machine.
*
* @return A new resource state machine.
*/
protected abstract ResourceType resourceType();
/**
* Returns the next server address.
*
* @return The next server address.
*/
private Address nextAddress() {
Address address = new Address("localhost", port++);
members.add(address);
return address;
}
/**
* Creates a set of Copycat servers.
*/
protected List<CopycatServer> createCopycatServers(int nodes) throws Throwable {
CountDownLatch latch = new CountDownLatch(nodes);
List<CopycatServer> servers = new ArrayList<>();
List<Address> members = new ArrayList<>();
for (int i = 0; i < nodes; i++) {
members.add(nextAddress());
}
for (int i = 0; i < nodes; i++) {
CopycatServer server = createCopycatServer(members.get(i));
server.open().thenRun(latch::countDown);
servers.add(server);
}
Uninterruptibles.awaitUninterruptibly(latch);
return servers;
}
/**
* Creates a Copycat server.
*/
protected CopycatServer createCopycatServer(Address address) {
ResourceRegistry resourceRegistry = new ResourceRegistry();
resourceRegistry.register(resourceType());
CopycatServer server = CopycatServer.builder(address, members)
.withTransport(new LocalTransport(registry))
.withStorage(Storage.builder()
.withStorageLevel(StorageLevel.DISK)
.withDirectory(TEST_DIR + "/" + address.port())
.withSerializer(serializer.clone())
.build())
.withStateMachine(() -> new ResourceManagerState(resourceRegistry))
.withSerializer(serializer.clone())
.withHeartbeatInterval(Duration.ofMillis(25))
.withElectionTimeout(Duration.ofMillis(50))
.withSessionTimeout(Duration.ofMillis(100))
.build();
copycatServers.add(server);
return server;
}
@Before
@After
public void clearTests() throws Exception {
registry = new LocalServerRegistry();
members = new ArrayList<>();
port = 5000;
CompletableFuture<Void> closeClients =
CompletableFuture.allOf(atomixClients.stream()
.map(Atomix::close)
.toArray(CompletableFuture[]::new));
closeClients.thenCompose(v -> CompletableFuture.allOf(copycatServers.stream()
.map(CopycatServer::close)
.toArray(CompletableFuture[]::new))).join();
deleteDirectory(TEST_DIR);
atomixClients = new ArrayList<>();
copycatServers = new ArrayList<>();
}
/**
* Deletes a directory recursively.
*/
private void deleteDirectory(File directory) throws IOException {
if (directory.exists()) {
File[] files = directory.listFiles();
if (files != null) {
for (File file : files) {
if (file.isDirectory()) {
deleteDirectory(file);
} else {
Files.delete(file.toPath());
}
}
}
Files.delete(directory.toPath());
}
}
/**
* Creates a Atomix client.
*/
protected Atomix createAtomixClient() {
CountDownLatch latch = new CountDownLatch(1);
Atomix client = AtomixClient.builder(members)
.withTransport(new LocalTransport(registry))
.withSerializer(serializer.clone())
.withResourceResolver(r -> r.register(resourceType()))
.build();
client.open().thenRun(latch::countDown);
atomixClients.add(client);
Uninterruptibles.awaitUninterruptibly(latch);
return client;
}
}