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
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.impl;
17 +
18 +import io.atomix.catalyst.serializer.CatalystSerializable;
19 +import io.atomix.catalyst.serializer.Serializer;
20 +import io.atomix.catalyst.serializer.TypeSerializerFactory;
21 +import io.atomix.copycat.client.Query;
22 +import io.atomix.manager.state.GetResource;
23 +import io.atomix.manager.state.GetResourceKeys;
24 +import io.atomix.resource.ResourceQuery;
25 +
26 +import java.io.IOException;
27 +import java.net.URL;
28 +import java.util.Arrays;
29 +import java.util.Enumeration;
30 +import java.util.Scanner;
31 +
32 +import org.onlab.util.Match;
33 +import org.onosproject.cluster.NodeId;
34 +import org.onosproject.event.Change;
35 +import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands;
36 +import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapState;
37 +import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands;
38 +import org.onosproject.store.primitives.resources.impl.CommitResult;
39 +import org.onosproject.store.primitives.resources.impl.MapEntryUpdateResult;
40 +import org.onosproject.store.primitives.resources.impl.MapUpdate;
41 +import org.onosproject.store.primitives.resources.impl.PrepareResult;
42 +import org.onosproject.store.primitives.resources.impl.RollbackResult;
43 +import org.onosproject.store.primitives.resources.impl.TransactionId;
44 +import org.onosproject.store.primitives.resources.impl.TransactionalMapUpdate;
45 +import org.onosproject.store.serializers.KryoNamespaces;
46 +import org.onosproject.store.service.MapEvent;
47 +import org.onosproject.store.service.Versioned;
48 +
49 +import com.google.common.base.Throwables;
50 +import com.google.common.collect.Maps;
51 +
52 +/**
53 + * Serializer utility for Atomix Catalyst.
54 + */
55 +public final class CatalystSerializers {
56 +
57 + private CatalystSerializers() {
58 + }
59 +
60 + public static Serializer getSerializer() {
61 + Serializer serializer = new Serializer();
62 + TypeSerializerFactory factory =
63 + new DefaultCatalystTypeSerializerFactory(
64 + org.onosproject.store.service.Serializer.using(Arrays.asList((KryoNamespaces.API)),
65 + MapEntryUpdateResult.class,
66 + MapEntryUpdateResult.Status.class,
67 + MapUpdate.class,
68 + MapUpdate.Type.class,
69 + TransactionalMapUpdate.class,
70 + TransactionId.class,
71 + PrepareResult.class,
72 + CommitResult.class,
73 + RollbackResult.class,
74 + AtomixConsistentMapCommands.Get.class,
75 + AtomixConsistentMapCommands.ContainsKey.class,
76 + AtomixConsistentMapCommands.ContainsValue.class,
77 + AtomixConsistentMapCommands.Size.class,
78 + AtomixConsistentMapCommands.IsEmpty.class,
79 + AtomixConsistentMapCommands.KeySet.class,
80 + AtomixConsistentMapCommands.EntrySet.class,
81 + AtomixConsistentMapCommands.Values.class,
82 + AtomixConsistentMapCommands.UpdateAndGet.class,
83 + AtomixConsistentMapCommands.TransactionPrepare.class,
84 + AtomixConsistentMapCommands.TransactionCommit.class,
85 + AtomixConsistentMapCommands.TransactionRollback.class,
86 + AtomixLeaderElectorCommands.GetLeadership.class,
87 + AtomixLeaderElectorCommands.GetAllLeaderships.class,
88 + AtomixLeaderElectorCommands.GetElectedTopics.class,
89 + AtomixLeaderElectorCommands.Run.class,
90 + AtomixLeaderElectorCommands.Withdraw.class,
91 + AtomixLeaderElectorCommands.Anoint.class,
92 + GetResource.class,
93 + GetResourceKeys.class,
94 + ResourceQuery.class,
95 + Query.ConsistencyLevel.class));
96 + // ONOS classes
97 + serializer.register(Change.class, factory);
98 + serializer.register(NodeId.class, factory);
99 + serializer.register(Match.class, factory);
100 + serializer.register(MapEntryUpdateResult.class, factory);
101 + serializer.register(MapEntryUpdateResult.Status.class, factory);
102 + serializer.register(TransactionalMapUpdate.class, factory);
103 + serializer.register(PrepareResult.class, factory);
104 + serializer.register(CommitResult.class, factory);
105 + serializer.register(RollbackResult.class, factory);
106 + serializer.register(TransactionId.class, factory);
107 + serializer.register(MapUpdate.class, factory);
108 + serializer.register(Versioned.class, factory);
109 + serializer.register(MapEvent.class, factory);
110 + serializer.register(Maps.immutableEntry("a", "b").getClass(), factory);
111 + serializer.register(AtomixConsistentMapState.class, factory);
112 +
113 + serializer.register(ResourceQuery.class, factory);
114 + serializer.register(GetResource.class, factory);
115 + serializer.register(GetResourceKeys.class, factory);
116 +
117 + // ConsistentMap
118 + serializer.register(AtomixConsistentMapCommands.UpdateAndGet.class, factory);
119 + serializer.register(AtomixConsistentMapCommands.Clear.class);
120 + serializer.register(AtomixConsistentMapCommands.Listen.class);
121 + serializer.register(AtomixConsistentMapCommands.Unlisten.class);
122 + serializer.register(AtomixConsistentMapCommands.Get.class);
123 + serializer.register(AtomixConsistentMapCommands.ContainsKey.class);
124 + serializer.register(AtomixConsistentMapCommands.ContainsValue.class);
125 + serializer.register(AtomixConsistentMapCommands.EntrySet.class);
126 + serializer.register(AtomixConsistentMapCommands.IsEmpty.class);
127 + serializer.register(AtomixConsistentMapCommands.KeySet.class);
128 + serializer.register(AtomixConsistentMapCommands.Size.class);
129 + serializer.register(AtomixConsistentMapCommands.Values.class);
130 + serializer.register(AtomixConsistentMapCommands.TransactionPrepare.class);
131 + serializer.register(AtomixConsistentMapCommands.TransactionCommit.class);
132 + serializer.register(AtomixConsistentMapCommands.TransactionRollback.class);
133 + // LeaderElector
134 + serializer.register(AtomixLeaderElectorCommands.Run.class, factory);
135 + serializer.register(AtomixLeaderElectorCommands.Withdraw.class, factory);
136 + serializer.register(AtomixLeaderElectorCommands.Anoint.class, factory);
137 + serializer.register(AtomixLeaderElectorCommands.GetElectedTopics.class, factory);
138 + serializer.register(AtomixLeaderElectorCommands.GetElectedTopics.class, factory);
139 + serializer.register(AtomixLeaderElectorCommands.GetLeadership.class, factory);
140 + serializer.register(AtomixLeaderElectorCommands.GetAllLeaderships.class, factory);
141 + serializer.register(AtomixLeaderElectorCommands.Listen.class);
142 + serializer.register(AtomixLeaderElectorCommands.Unlisten.class);
143 + // Atomix types
144 + try {
145 + ClassLoader cl = CatalystSerializable.class.getClassLoader();
146 + Enumeration<URL> urls = cl.getResources(
147 + String.format("META-INF/services/%s", CatalystSerializable.class.getName()));
148 + while (urls.hasMoreElements()) {
149 + URL url = urls.nextElement();
150 + try (Scanner scanner = new Scanner(url.openStream(), "UTF-8")) {
151 + scanner.useDelimiter("\n").forEachRemaining(line -> {
152 + if (!line.trim().startsWith("#")) {
153 + line = line.trim();
154 + if (line.length() > 0) {
155 + try {
156 + serializer.register(cl.loadClass(line));
157 + } catch (ClassNotFoundException e) {
158 + Throwables.propagate(e);
159 + }
160 + }
161 + }
162 + });
163 + }
164 + }
165 + } catch (IOException e) {
166 + Throwables.propagate(e);
167 + }
168 + return serializer;
169 + }
170 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import io.atomix.variables.DistributedLong;
19 +
20 +import java.util.concurrent.CompletableFuture;
21 +
22 +import org.onosproject.store.service.AsyncAtomicCounter;
23 +
24 +/**
25 + * {@code AsyncAtomicCounter} implementation backed by Atomix
26 + * {@link DistributedLong}.
27 + */
28 +public class AtomixCounter implements AsyncAtomicCounter {
29 +
30 + private final String name;
31 + private final DistributedLong distLong;
32 +
33 + public AtomixCounter(String name, DistributedLong distLong) {
34 + this.name = name;
35 + this.distLong = distLong;
36 + }
37 +
38 + @Override
39 + public String name() {
40 + return name;
41 + }
42 +
43 + @Override
44 + public CompletableFuture<Long> incrementAndGet() {
45 + return distLong.incrementAndGet();
46 + }
47 +
48 + @Override
49 + public CompletableFuture<Long> getAndIncrement() {
50 + return distLong.getAndIncrement();
51 + }
52 +
53 + @Override
54 + public CompletableFuture<Long> getAndAdd(long delta) {
55 + return distLong.getAndAdd(delta);
56 + }
57 +
58 + @Override
59 + public CompletableFuture<Long> addAndGet(long delta) {
60 + return distLong.addAndGet(delta);
61 + }
62 +
63 + @Override
64 + public CompletableFuture<Long> get() {
65 + return distLong.get();
66 + }
67 +
68 + @Override
69 + public CompletableFuture<Void> set(long value) {
70 + return distLong.set(value);
71 + }
72 +
73 + @Override
74 + public CompletableFuture<Boolean> compareAndSet(long expectedValue,
75 + long updateValue) {
76 + return distLong.compareAndSet(expectedValue, updateValue);
77 + }
78 +}
...\ No newline at end of file ...\ No newline at end of file
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import io.atomix.catalyst.util.Listener;
19 +import io.atomix.copycat.client.CopycatClient;
20 +import io.atomix.resource.Consistency;
21 +import io.atomix.resource.Resource;
22 +import io.atomix.resource.ResourceTypeInfo;
23 +
24 +import java.util.Map;
25 +import java.util.Set;
26 +import java.util.concurrent.CompletableFuture;
27 +import java.util.function.Consumer;
28 +
29 +import org.onosproject.cluster.Leadership;
30 +import org.onosproject.cluster.NodeId;
31 +import org.onosproject.event.Change;
32 +import org.onosproject.store.service.AsyncLeaderElector;
33 +
34 +import com.google.common.collect.Sets;
35 +
36 +/**
37 + * Distributed resource providing the {@link AsyncLeaderElector} primitive.
38 + */
39 +@ResourceTypeInfo(id = -152, stateMachine = AtomixLeaderElectorState.class)
40 +public class AtomixLeaderElector
41 + extends Resource<AtomixLeaderElector, Resource.Options> implements AsyncLeaderElector {
42 + private final Set<Consumer<Change<Leadership>>> leadershipChangeListeners =
43 + Sets.newConcurrentHashSet();
44 +
45 + private Listener<Change<Leadership>> listener;
46 +
47 + public AtomixLeaderElector(CopycatClient client, Resource.Options options) {
48 + super(client, options);
49 + }
50 +
51 + @Override
52 + public String name() {
53 + return null;
54 + }
55 +
56 + @Override
57 + public CompletableFuture<AtomixLeaderElector> open() {
58 + return super.open().thenApply(result -> {
59 + client.session().onEvent("change", this::handleEvent);
60 + return result;
61 + });
62 + }
63 +
64 + private void handleEvent(Change<Leadership> change) {
65 + leadershipChangeListeners.forEach(l -> l.accept(change));
66 + }
67 +
68 + @Override
69 + public AtomixLeaderElector with(Consistency consistency) {
70 + super.with(consistency);
71 + return this;
72 + }
73 +
74 + @Override
75 + public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
76 + return submit(new AtomixLeaderElectorCommands.Run(topic, nodeId));
77 + }
78 +
79 + @Override
80 + public CompletableFuture<Void> withdraw(String topic) {
81 + return submit(new AtomixLeaderElectorCommands.Withdraw(topic));
82 + }
83 +
84 + @Override
85 + public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
86 + return submit(new AtomixLeaderElectorCommands.Anoint(topic, nodeId));
87 + }
88 +
89 + @Override
90 + public CompletableFuture<Leadership> getLeadership(String topic) {
91 + return submit(new AtomixLeaderElectorCommands.GetLeadership(topic));
92 + }
93 +
94 + @Override
95 + public CompletableFuture<Map<String, Leadership>> getLeaderships() {
96 + return submit(new AtomixLeaderElectorCommands.GetAllLeaderships());
97 + }
98 +
99 + public CompletableFuture<Set<String>> getElectedTopics(NodeId nodeId) {
100 + return submit(new AtomixLeaderElectorCommands.GetElectedTopics(nodeId));
101 + }
102 +
103 + /**
104 + * Leadership change listener context.
105 + */
106 + private final class LeadershipChangeListener implements Listener<Change<Leadership>> {
107 + private final Consumer<Change<Leadership>> listener;
108 +
109 + private LeadershipChangeListener(Consumer<Change<Leadership>> listener) {
110 + this.listener = listener;
111 + }
112 +
113 + @Override
114 + public void accept(Change<Leadership> change) {
115 + listener.accept(change);
116 + }
117 +
118 + @Override
119 + public void close() {
120 + synchronized (AtomixLeaderElector.this) {
121 + submit(new AtomixLeaderElectorCommands.Unlisten());
122 + }
123 + }
124 + }
125 +
126 + @Override
127 + public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
128 + leadershipChangeListeners.add(consumer);
129 + return setupListener();
130 + }
131 +
132 + @Override
133 + public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
134 + leadershipChangeListeners.remove(consumer);
135 + return teardownListener();
136 + }
137 +
138 + private CompletableFuture<Void> setupListener() {
139 + if (listener == null && !leadershipChangeListeners.isEmpty()) {
140 + Consumer<Change<Leadership>> changeConsumer = change -> {
141 + leadershipChangeListeners.forEach(consumer -> consumer.accept(change));
142 + };
143 + return submit(new AtomixLeaderElectorCommands.Listen())
144 + .thenAccept(v -> listener = new LeadershipChangeListener(changeConsumer));
145 + }
146 + return CompletableFuture.completedFuture(null);
147 + }
148 +
149 + private CompletableFuture<Void> teardownListener() {
150 + if (listener != null && leadershipChangeListeners.isEmpty()) {
151 + listener.close();
152 + listener = null;
153 + return submit(new AtomixLeaderElectorCommands.Unlisten());
154 + }
155 + return CompletableFuture.completedFuture(null);
156 + }
157 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import java.util.Map;
19 +import java.util.Set;
20 +
21 +import org.onosproject.cluster.Leadership;
22 +import org.onosproject.cluster.NodeId;
23 +
24 +import com.google.common.base.MoreObjects;
25 +import com.google.common.base.Strings;
26 +
27 +import io.atomix.catalyst.buffer.BufferInput;
28 +import io.atomix.catalyst.buffer.BufferOutput;
29 +import io.atomix.catalyst.serializer.CatalystSerializable;
30 +import io.atomix.catalyst.serializer.Serializer;
31 +import io.atomix.catalyst.util.Assert;
32 +import io.atomix.copycat.client.Command;
33 +import io.atomix.copycat.client.Query;
34 +
35 +/**
36 + * {@link AtomixLeaderElector} resource state machine operations.
37 + */
38 +public final class AtomixLeaderElectorCommands {
39 +
40 + private AtomixLeaderElectorCommands() {
41 + }
42 +
43 + /**
44 + * Abstract election query.
45 + */
46 + @SuppressWarnings("serial")
47 + public abstract static class ElectionQuery<V> implements Query<V>, CatalystSerializable {
48 +
49 + @Override
50 + public ConsistencyLevel consistency() {
51 + return ConsistencyLevel.BOUNDED_LINEARIZABLE;
52 + }
53 +
54 + @Override
55 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
56 + }
57 +
58 + @Override
59 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
60 + }
61 + }
62 +
63 + /**
64 + * Abstract election topic query.
65 + */
66 + @SuppressWarnings("serial")
67 + public abstract static class TopicQuery<V> extends ElectionQuery<V> implements CatalystSerializable {
68 + String topic;
69 +
70 + public TopicQuery() {
71 + }
72 +
73 + public TopicQuery(String topic) {
74 + this.topic = Assert.notNull(topic, "topic");
75 + }
76 +
77 + /**
78 + * Returns the topic.
79 + * @return topic
80 + */
81 + public String topic() {
82 + return topic;
83 + }
84 +
85 + @Override
86 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
87 + serializer.writeObject(topic, buffer);
88 + }
89 +
90 + @Override
91 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
92 + topic = serializer.readObject(buffer);
93 + }
94 + }
95 +
96 + /**
97 + * Abstract election command.
98 + */
99 + @SuppressWarnings("serial")
100 + public abstract static class ElectionCommand<V> implements Command<V>, CatalystSerializable {
101 +
102 + @Override
103 + public ConsistencyLevel consistency() {
104 + return ConsistencyLevel.LINEARIZABLE;
105 + }
106 +
107 + @Override
108 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
109 + }
110 +
111 + @Override
112 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
113 + }
114 + }
115 +
116 + /**
117 + * Listen command.
118 + */
119 + @SuppressWarnings("serial")
120 + public static class Listen extends ElectionCommand<Void> {
121 + }
122 +
123 + /**
124 + * Unlisten command.
125 + */
126 + @SuppressWarnings("serial")
127 + public static class Unlisten extends ElectionCommand<Void> {
128 +
129 + @Override
130 + public CompactionMode compaction() {
131 + return CompactionMode.QUORUM;
132 + }
133 + }
134 +
135 + /**
136 + * GetLeader query.
137 + */
138 + @SuppressWarnings("serial")
139 + public static class GetLeadership extends TopicQuery<Leadership> {
140 +
141 + public GetLeadership() {
142 + }
143 +
144 + public GetLeadership(String topic) {
145 + super(topic);
146 + }
147 +
148 + @Override
149 + public String toString() {
150 + return MoreObjects.toStringHelper(getClass())
151 + .add("topic", topic)
152 + .toString();
153 + }
154 + }
155 +
156 + /**
157 + * GetAllLeaders query.
158 + */
159 + @SuppressWarnings("serial")
160 + public static class GetAllLeaderships extends ElectionQuery<Map<String, Leadership>> {
161 + }
162 +
163 + /**
164 + * GetElectedTopics query.
165 + */
166 + @SuppressWarnings("serial")
167 + public static class GetElectedTopics extends ElectionQuery<Set<String>> {
168 + private NodeId nodeId;
169 +
170 + public GetElectedTopics() {
171 + }
172 +
173 + public GetElectedTopics(NodeId nodeId) {
174 + this.nodeId = Assert.argNot(nodeId, nodeId == null, "nodeId cannot be null");
175 + }
176 +
177 + /**
178 + * Returns the nodeId to check.
179 + *
180 + * @return The nodeId to check.
181 + */
182 + public NodeId nodeId() {
183 + return nodeId;
184 + }
185 +
186 + @Override
187 + public String toString() {
188 + return MoreObjects.toStringHelper(getClass())
189 + .add("nodeId", nodeId)
190 + .toString();
191 + }
192 + }
193 +
194 + /**
195 + * Enter and run for leadership.
196 + */
197 + @SuppressWarnings("serial")
198 + public static class Run extends ElectionCommand<Leadership> {
199 + private String topic;
200 + private NodeId nodeId;
201 +
202 + public Run() {
203 + }
204 +
205 + public Run(String topic, NodeId nodeId) {
206 + this.topic = Assert.argNot(topic, Strings.isNullOrEmpty(topic), "topic cannot be null or empty");
207 + this.nodeId = Assert.argNot(nodeId, nodeId == null, "nodeId cannot be null");
208 + }
209 +
210 + /**
211 + * Returns the topic.
212 + *
213 + * @return topic
214 + */
215 + public String topic() {
216 + return topic;
217 + }
218 +
219 + /**
220 + * Returns the nodeId.
221 + *
222 + * @return the nodeId
223 + */
224 + public NodeId nodeId() {
225 + return nodeId;
226 + }
227 +
228 + @Override
229 + public String toString() {
230 + return MoreObjects.toStringHelper(getClass())
231 + .add("topic", topic)
232 + .add("nodeId", nodeId)
233 + .toString();
234 + }
235 + }
236 +
237 + /**
238 + * Withdraw from a leadership contest.
239 + */
240 + @SuppressWarnings("serial")
241 + public static class Withdraw extends ElectionCommand<Void> {
242 + private String topic;
243 +
244 + public Withdraw() {
245 + }
246 +
247 + public Withdraw(String topic) {
248 + this.topic = Assert.argNot(topic, Strings.isNullOrEmpty(topic), "topic cannot be null or empty");
249 + }
250 +
251 + /**
252 + * Returns the topic.
253 + *
254 + * @return The topic
255 + */
256 + public String topic() {
257 + return topic;
258 + }
259 +
260 + @Override
261 + public String toString() {
262 + return MoreObjects.toStringHelper(getClass())
263 + .add("topic", topic)
264 + .toString();
265 + }
266 + }
267 +
268 + /**
269 + * Command for administratively anointing a node as leader.
270 + */
271 + @SuppressWarnings("serial")
272 + public static class Anoint extends ElectionCommand<Boolean> {
273 + private String topic;
274 + private NodeId nodeId;
275 +
276 + public Anoint() {
277 + }
278 +
279 + public Anoint(String topic, NodeId nodeId) {
280 + this.topic = topic;
281 + this.nodeId = nodeId;
282 + }
283 +
284 + /**
285 + * Returns the topic.
286 + *
287 + * @return The topic
288 + */
289 + public String topic() {
290 + return topic;
291 + }
292 +
293 + /**
294 + * Returns the nodeId to make leader.
295 + *
296 + * @return The nodeId
297 + */
298 + public NodeId nodeId() {
299 + return nodeId;
300 + }
301 +
302 + @Override
303 + public String toString() {
304 + return MoreObjects.toStringHelper(getClass())
305 + .add("topic", topic)
306 + .add("nodeId", nodeId)
307 + .toString();
308 + }
309 + }
310 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import java.util.Objects;
19 +import java.util.Set;
20 +import java.util.concurrent.CompletableFuture;
21 +
22 +import org.onlab.util.Tools;
23 +import org.onosproject.store.service.AsyncAtomicValue;
24 +import org.onosproject.store.service.AtomicValueEvent;
25 +import org.onosproject.store.service.AtomicValueEventListener;
26 +import org.onosproject.store.service.MapEventListener;
27 +import org.onosproject.store.service.Versioned;
28 +
29 +import com.google.common.collect.Sets;
30 +
31 +/**
32 + * Implementation of {@link AsyncAtomicValue} backed by {@link AtomixConsistentMap}.
33 + */
34 +public class AtomixValue implements AsyncAtomicValue<String> {
35 +
36 + private final String name;
37 + private final AtomixConsistentMap atomixMap;
38 + private MapEventListener<String, byte[]> mapEventListener;
39 + private final Set<AtomicValueEventListener<String>> listeners = Sets.newIdentityHashSet();
40 +
41 + AtomixValue(String name, AtomixConsistentMap atomixMap) {
42 + this.name = name;
43 + this.atomixMap = atomixMap;
44 + }
45 +
46 + @Override
47 + public CompletableFuture<Boolean> compareAndSet(String expect, String update) {
48 + return atomixMap.replace(name, Tools.getBytesUtf8(expect), Tools.getBytesUtf8(update));
49 + }
50 +
51 + @Override
52 + public CompletableFuture<String> get() {
53 + return atomixMap.get(name)
54 + .thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
55 + }
56 +
57 + @Override
58 + public CompletableFuture<String> getAndSet(String value) {
59 + return atomixMap.put(name, Tools.getBytesUtf8(value))
60 + .thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
61 + }
62 +
63 + @Override
64 + public CompletableFuture<Void> set(String value) {
65 + return getAndSet(value).thenApply(v -> null);
66 + }
67 +
68 + @Override
69 + public CompletableFuture<Void> addListener(AtomicValueEventListener<String> listener) {
70 + // TODO: synchronization
71 + if (mapEventListener == null) {
72 + mapEventListener = event -> {
73 + Versioned<byte[]> newValue = event.newValue();
74 + Versioned<byte[]> oldValue = event.oldValue();
75 + if (Objects.equals(event.key(), name)) {
76 + listener.event(new AtomicValueEvent<>(name,
77 + newValue == null ? null : Tools.toStringUtf8(newValue.value()),
78 + oldValue == null ? null : Tools.toStringUtf8(oldValue.value())));
79 + }
80 + };
81 + return atomixMap.addListener(mapEventListener).whenComplete((r, e) -> {
82 + if (e == null) {
83 + listeners.add(listener);
84 + } else {
85 + mapEventListener = null;
86 + }
87 + });
88 + } else {
89 + listeners.add(listener);
90 + return CompletableFuture.completedFuture(null);
91 + }
92 + }
93 +
94 + @Override
95 + public CompletableFuture<Void> removeListener(AtomicValueEventListener<String> listener) {
96 + // TODO: synchronization
97 + listeners.remove(listener);
98 + if (listeners.isEmpty()) {
99 + return atomixMap.removeListener(mapEventListener);
100 + } else {
101 + return CompletableFuture.completedFuture(null);
102 + }
103 + }
104 +
105 + @Override
106 + public String name() {
107 + return null;
108 + }
109 +}
...\ No newline at end of file ...\ No newline at end of file
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +/**
19 + * Response enum for two phase commit operation.
20 + */
21 +public enum CommitResult {
22 + /**
23 + * Signifies a successful commit execution.
24 + */
25 + OK,
26 +
27 + /**
28 + * Signifies a failure due to unrecognized transaction identifier.
29 + */
30 + UNKNOWN_TRANSACTION_ID,
31 +}
1 +
2 +/*
3 + * Copyright 2015 Open Networking Laboratory
4 + *
5 + * Licensed under the Apache License, Version 2.0 (the "License");
6 + * you may not use this file except in compliance with the License.
7 + * You may obtain a copy of the License at
8 + *
9 + * http://www.apache.org/licenses/LICENSE-2.0
10 + *
11 + * Unless required by applicable law or agreed to in writing, software
12 + * distributed under the License is distributed on an "AS IS" BASIS,
13 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 + * See the License for the specific language governing permissions and
15 + * limitations under the License.
16 + */
17 +package org.onosproject.store.primitives.resources.impl;
18 +
19 +import java.util.function.Function;
20 +
21 +import org.onosproject.store.service.MapEvent;
22 +import org.onosproject.store.service.Versioned;
23 +
24 +import com.google.common.base.MoreObjects;
25 +
26 +/**
27 + * Result of a map entry update operation.
28 + * <p>
29 + * Both old and new values are accessible along with a flag that indicates if the
30 + * the value was updated. If flag is false, oldValue and newValue both
31 + * point to the same unmodified value.
32 + * @param <V> result type
33 + */
34 +public class MapEntryUpdateResult<K, V> {
35 +
36 + public enum Status {
37 +
38 + /**
39 + * Indicates a successful update.
40 + */
41 + OK,
42 +
43 + /**
44 + * Indicates a noop i.e. existing and new value are both null.
45 + */
46 + NOOP,
47 +
48 + /**
49 + * Indicates a failed update due to a write lock.
50 + */
51 + WRITE_LOCK,
52 +
53 + /**
54 + * Indicates a failed update due to a precondition check failure.
55 + */
56 + PRECONDITION_FAILED
57 + }
58 +
59 + private final String mapName;
60 + private Status status;
61 + private final K key;
62 + private final Versioned<V> oldValue;
63 + private final Versioned<V> newValue;
64 +
65 + public MapEntryUpdateResult(Status status, String mapName, K key, Versioned<V> oldValue, Versioned<V> newValue) {
66 + this.status = status;
67 + this.mapName = mapName;
68 + this.key = key;
69 + this.oldValue = oldValue;
70 + this.newValue = newValue;
71 + }
72 +
73 + /**
74 + * Returns {@code true} if the update was successful.
75 + * @return {@code true} if yes, {@code false} otherwise
76 + */
77 + public boolean updated() {
78 + return status == Status.OK;
79 + }
80 +
81 + /**
82 + * Returns the map name.
83 + * @return map name
84 + */
85 + public String mapName() {
86 + return mapName;
87 + }
88 +
89 + /**
90 + * Returns the update status.
91 + * @return update status
92 + */
93 + public Status status() {
94 + return status;
95 + }
96 +
97 + /**
98 + * Returns the map key.
99 + * @return key
100 + */
101 + public K key() {
102 + return key;
103 + }
104 +
105 + /**
106 + * Returns the old value.
107 + * @return the previous value associated with key if updated was successful, otherwise current value
108 + */
109 + public Versioned<V> oldValue() {
110 + return oldValue;
111 + }
112 +
113 + /**
114 + * Returns the new value after update.
115 + * @return if updated was unsuccessful, this is same as old value
116 + */
117 + public Versioned<V> newValue() {
118 + return newValue;
119 + }
120 +
121 + /**
122 + * Maps to another instance with different key and value types.
123 + * @param keyTransform transformer to use for transcoding keys
124 + * @param valueMapper mapper to use for transcoding values
125 + * @return new instance
126 + */
127 + public <K1, V1> MapEntryUpdateResult<K1, V1> map(Function<K, K1> keyTransform, Function<V, V1> valueMapper) {
128 + return new MapEntryUpdateResult<>(status,
129 + mapName,
130 + keyTransform.apply(key),
131 + oldValue == null ? null : oldValue.map(valueMapper),
132 + newValue == null ? null : newValue.map(valueMapper));
133 + }
134 +
135 + /**
136 + * Return the map event that will be generated as a result of this update.
137 + * @return map event. if update was unsuccessful, this returns {@code null}
138 + */
139 + public MapEvent<K, V> toMapEvent() {
140 + if (!updated()) {
141 + return null;
142 + } else {
143 + return new MapEvent<>(mapName(), key(), newValue, oldValue);
144 + }
145 + }
146 +
147 + @Override
148 + public String toString() {
149 + return MoreObjects.toStringHelper(MapEntryUpdateResult.class)
150 + .add("mapName", mapName)
151 + .add("status", status)
152 + .add("key", key)
153 + .add("oldValue", oldValue)
154 + .add("newValue", newValue)
155 + .toString();
156 + }
157 +}
1 +/*
2 + * Copyright 2015 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +
17 +package org.onosproject.store.primitives.resources.impl;
18 +
19 +import static com.google.common.base.Preconditions.checkArgument;
20 +import static com.google.common.base.Preconditions.checkNotNull;
21 +import static com.google.common.base.Preconditions.checkState;
22 +
23 +import com.google.common.base.MoreObjects;
24 +
25 +/**
26 + * Map update operation.
27 + *
28 + * @param <K> map key type
29 + * @param <V> map value type
30 + *
31 + */
32 +public final class MapUpdate<K, V> {
33 +
34 + /**
35 + * Type of database update operation.
36 + */
37 + public enum Type {
38 + /**
39 + * Insert/Update entry without any checks.
40 + */
41 + PUT,
42 + /**
43 + * Insert an entry iff there is no existing entry for that key.
44 + */
45 + PUT_IF_ABSENT,
46 +
47 + /**
48 + * Update entry if the current version matches specified version.
49 + */
50 + PUT_IF_VERSION_MATCH,
51 +
52 + /**
53 + * Update entry if the current value matches specified value.
54 + */
55 + PUT_IF_VALUE_MATCH,
56 +
57 + /**
58 + * Remove entry without any checks.
59 + */
60 + REMOVE,
61 +
62 + /**
63 + * Remove entry if the current version matches specified version.
64 + */
65 + REMOVE_IF_VERSION_MATCH,
66 +
67 + /**
68 + * Remove entry if the current value matches specified value.
69 + */
70 + REMOVE_IF_VALUE_MATCH,
71 + }
72 +
73 + private Type type;
74 + private K key;
75 + private V value;
76 + private V currentValue;
77 + private long currentVersion = -1;
78 +
79 + /**
80 + * Returns the type of update operation.
81 + * @return type of update.
82 + */
83 + public Type type() {
84 + return type;
85 + }
86 +
87 + /**
88 + * Returns the item key being updated.
89 + * @return item key
90 + */
91 + public K key() {
92 + return key;
93 + }
94 +
95 + /**
96 + * Returns the new value.
97 + * @return item's target value.
98 + */
99 + public V value() {
100 + return value;
101 + }
102 +
103 + /**
104 + * Returns the expected current value for the key.
105 + * @return current value in database.
106 + */
107 + public V currentValue() {
108 + return currentValue;
109 + }
110 +
111 + /**
112 + * Returns the expected current version in the database for the key.
113 + * @return expected version.
114 + */
115 + public long currentVersion() {
116 + return currentVersion;
117 + }
118 +
119 + @Override
120 + public String toString() {
121 + return MoreObjects.toStringHelper(this)
122 + .add("type", type)
123 + .add("key", key)
124 + .add("value", value)
125 + .add("currentValue", currentValue)
126 + .add("currentVersion", currentVersion)
127 + .toString();
128 + }
129 +
130 + /**
131 + * Creates a new builder instance.
132 + *
133 + * @param <K> key type
134 + * @param <V> value type
135 + * @return builder.
136 + */
137 + public static <K, V> Builder<K, V> newBuilder() {
138 + return new Builder<>();
139 + }
140 +
141 + /**
142 + * MapUpdate builder.
143 + *
144 + * @param <K> key type
145 + * @param <V> value type
146 + */
147 + public static final class Builder<K, V> {
148 +
149 + private MapUpdate<K, V> update = new MapUpdate<>();
150 +
151 + public MapUpdate<K, V> build() {
152 + validateInputs();
153 + return update;
154 + }
155 +
156 + public Builder<K, V> withType(Type type) {
157 + update.type = checkNotNull(type, "type cannot be null");
158 + return this;
159 + }
160 +
161 + public Builder<K, V> withKey(K key) {
162 + update.key = checkNotNull(key, "key cannot be null");
163 + return this;
164 + }
165 +
166 + public Builder<K, V> withCurrentValue(V value) {
167 + update.currentValue = checkNotNull(value, "currentValue cannot be null");
168 + return this;
169 + }
170 +
171 + public Builder<K, V> withValue(V value) {
172 + update.value = checkNotNull(value, "value cannot be null");
173 + return this;
174 + }
175 +
176 + public Builder<K, V> withCurrentVersion(long version) {
177 + checkArgument(version >= 0, "version cannot be negative");
178 + update.currentVersion = version;
179 + return this;
180 + }
181 +
182 + private void validateInputs() {
183 + checkNotNull(update.type, "type must be specified");
184 + checkNotNull(update.key, "key must be specified");
185 + switch (update.type) {
186 + case PUT:
187 + case PUT_IF_ABSENT:
188 + checkNotNull(update.value, "value must be specified.");
189 + break;
190 + case PUT_IF_VERSION_MATCH:
191 + checkNotNull(update.value, "value must be specified.");
192 + checkState(update.currentVersion >= 0, "current version must be specified");
193 + break;
194 + case PUT_IF_VALUE_MATCH:
195 + checkNotNull(update.value, "value must be specified.");
196 + checkNotNull(update.currentValue, "currentValue must be specified.");
197 + break;
198 + case REMOVE:
199 + break;
200 + case REMOVE_IF_VERSION_MATCH:
201 + checkState(update.currentVersion >= 0, "current version must be specified");
202 + break;
203 + case REMOVE_IF_VALUE_MATCH:
204 + checkNotNull(update.currentValue, "currentValue must be specified.");
205 + break;
206 + default:
207 + throw new IllegalStateException("Unknown operation type");
208 + }
209 + }
210 + }
211 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +/**
19 + * Response enum for two phase commit prepare operation.
20 + */
21 +public enum PrepareResult {
22 + /**
23 + * Signifies a successful execution of the prepare operation.
24 + */
25 + OK,
26 +
27 + /**
28 + * Signifies a failure to another transaction locking the underlying state.
29 + */
30 + CONCURRENT_TRANSACTION,
31 +
32 + /**
33 + * Signifies a optimistic lock failure. This can happen if underlying state has changed since it was last read.
34 + */
35 + OPTIMISTIC_LOCK_FAILURE,
36 +}
...\ No newline at end of file ...\ No newline at end of file
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +/**
19 + * Response enum for two phase commit rollback operation.
20 + */
21 +public enum RollbackResult {
22 + /**
23 + * Signifies a successful rollback execution.
24 + */
25 + OK,
26 +
27 + /**
28 + * Signifies a failure due to unrecognized transaction identifier.
29 + */
30 + UNKNOWN_TRANSACTION_ID,
31 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import com.google.common.base.Objects;
19 +
20 +/**
21 + * Transaction identifier.
22 + */
23 +public final class TransactionId {
24 +
25 + public static TransactionId from(String id) {
26 + return new TransactionId(id);
27 + }
28 +
29 + private final String id;
30 +
31 + private TransactionId(String id) {
32 + this.id = id;
33 + }
34 +
35 + @Override
36 + public String toString() {
37 + return id;
38 + }
39 +
40 + @Override
41 + public int hashCode() {
42 + return id.hashCode();
43 + }
44 +
45 + @Override
46 + public boolean equals(Object other) {
47 + if (this == other) {
48 + return true;
49 + }
50 + if (other instanceof TransactionId) {
51 + TransactionId that = (TransactionId) other;
52 + return Objects.equal(this.id, that.id);
53 + }
54 + return false;
55 + }
56 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import java.util.Collection;
19 +import java.util.Map;
20 +
21 +import com.google.common.collect.ImmutableList;
22 +import com.google.common.collect.Maps;
23 +
24 +/**
25 + * A batch updates to an {@code AsyncConsistentMap} be committed as a transaction.
26 + *
27 + * @param <K> key type
28 + * @param <V> value type
29 + */
30 +public class TransactionalMapUpdate<K, V> {
31 + private final TransactionId transactionId;
32 + private final Collection<MapUpdate<K, V>> updates;
33 + private boolean indexPopulated = false;
34 + private final Map<K, V> keyValueIndex = Maps.newHashMap();
35 +
36 + public TransactionalMapUpdate(TransactionId transactionId, Collection<MapUpdate<K, V>> updates) {
37 + this.transactionId = transactionId;
38 + this.updates = ImmutableList.copyOf(updates);
39 + populateIndex();
40 + }
41 +
42 + /**
43 + * Returns the transaction identifier.
44 + * @return transaction id
45 + */
46 + public TransactionId transactionId() {
47 + return transactionId;
48 + }
49 +
50 + /**
51 + * Returns the collection of map updates.
52 + * @return map updates
53 + */
54 + public Collection<MapUpdate<K, V>> batch() {
55 + return updates;
56 + }
57 +
58 + /**
59 + * Returns the value that will be associated with the key after this transaction commits.
60 + * @param key key
61 + * @return value that will be associated with the value once this transaction commits
62 + */
63 + public V valueForKey(K key) {
64 + if (!indexPopulated) {
65 + // We do not synchronize as we don't expect this called to be made from multiple threads.
66 + populateIndex();
67 + }
68 + return keyValueIndex.get(key);
69 + }
70 +
71 + /**
72 + * Populates the internal key -> value mapping.
73 + */
74 + private synchronized void populateIndex() {
75 + updates.forEach(mapUpdate -> {
76 + if (mapUpdate.value() != null) {
77 + keyValueIndex.put(mapUpdate.key(), mapUpdate.value());
78 + }
79 + });
80 + indexPopulated = true;
81 + }
82 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +
17 +/**
18 + * State machine implementation classes for distributed primitives.
19 + */
20 +package org.onosproject.store.primitives.resources.impl;
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import static org.junit.Assert.*;
19 +
20 +import org.junit.Test;
21 +
22 +import io.atomix.Atomix;
23 +import io.atomix.resource.ResourceType;
24 +import io.atomix.variables.DistributedLong;
25 +
26 +/**
27 + * Unit tests for {@link AtomixCounter}.
28 + */
29 +public class AtomixLongTest extends AtomixTestBase {
30 +
31 + @Override
32 + protected ResourceType resourceType() {
33 + return new ResourceType(DistributedLong.class);
34 + }
35 +
36 + @Test
37 + public void testBasicOperations() throws Throwable {
38 + basicOperationsTest(1);
39 + clearTests();
40 + basicOperationsTest(2);
41 + clearTests();
42 + basicOperationsTest(3);
43 + clearTests();
44 + }
45 +
46 + protected void basicOperationsTest(int clusterSize) throws Throwable {
47 + createCopycatServers(clusterSize);
48 + Atomix atomix = createAtomixClient();
49 + AtomixCounter along = new AtomixCounter("test-long", atomix.getLong("test-long").join());
50 + assertEquals(0, along.get().join().longValue());
51 + assertEquals(1, along.incrementAndGet().join().longValue());
52 + along.set(100).join();
53 + assertEquals(100, along.get().join().longValue());
54 + assertEquals(100, along.getAndAdd(10).join().longValue());
55 + assertEquals(110, along.get().join().longValue());
56 + assertFalse(along.compareAndSet(109, 111).join());
57 + assertTrue(along.compareAndSet(110, 111).join());
58 + assertEquals(100, along.addAndGet(-11).join().longValue());
59 + assertEquals(100, along.getAndIncrement().join().longValue());
60 + assertEquals(101, along.get().join().longValue());
61 + }
62 +}
1 +/*
2 + * Copyright 2016 Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import io.atomix.Atomix;
19 +import io.atomix.AtomixClient;
20 +import io.atomix.catalyst.serializer.Serializer;
21 +import io.atomix.catalyst.transport.Address;
22 +import io.atomix.catalyst.transport.LocalServerRegistry;
23 +import io.atomix.catalyst.transport.LocalTransport;
24 +import io.atomix.copycat.client.CopycatClient;
25 +import io.atomix.copycat.server.CopycatServer;
26 +import io.atomix.copycat.server.storage.Storage;
27 +import io.atomix.copycat.server.storage.StorageLevel;
28 +import io.atomix.manager.state.ResourceManagerState;
29 +import io.atomix.resource.ResourceRegistry;
30 +import io.atomix.resource.ResourceType;
31 +
32 +import java.io.File;
33 +import java.io.IOException;
34 +import java.nio.file.Files;
35 +import java.time.Duration;
36 +import java.util.ArrayList;
37 +import java.util.List;
38 +import java.util.concurrent.CompletableFuture;
39 +import java.util.concurrent.CountDownLatch;
40 +
41 +import org.junit.After;
42 +import org.junit.Before;
43 +import org.onosproject.store.primitives.impl.CatalystSerializers;
44 +
45 +import com.google.common.util.concurrent.Uninterruptibles;
46 +
47 +/**
48 + * Base class for various Atomix* tests.
49 + */
50 +public abstract class AtomixTestBase {
51 + private static final File TEST_DIR = new File("target/test-logs");
52 + protected LocalServerRegistry registry;
53 + protected int port;
54 + protected List<Address> members;
55 + protected List<CopycatClient> copycatClients = new ArrayList<>();
56 + protected List<CopycatServer> copycatServers = new ArrayList<>();
57 + protected List<Atomix> atomixClients = new ArrayList<>();
58 + protected List<CopycatServer> atomixServers = new ArrayList<>();
59 + protected Serializer serializer = CatalystSerializers.getSerializer();
60 +
61 + /**
62 + * Creates a new resource state machine.
63 + *
64 + * @return A new resource state machine.
65 + */
66 + protected abstract ResourceType resourceType();
67 +
68 + /**
69 + * Returns the next server address.
70 + *
71 + * @return The next server address.
72 + */
73 + private Address nextAddress() {
74 + Address address = new Address("localhost", port++);
75 + members.add(address);
76 + return address;
77 + }
78 +
79 + /**
80 + * Creates a set of Copycat servers.
81 + */
82 + protected List<CopycatServer> createCopycatServers(int nodes) throws Throwable {
83 + CountDownLatch latch = new CountDownLatch(nodes);
84 + List<CopycatServer> servers = new ArrayList<>();
85 +
86 + List<Address> members = new ArrayList<>();
87 + for (int i = 0; i < nodes; i++) {
88 + members.add(nextAddress());
89 + }
90 +
91 + for (int i = 0; i < nodes; i++) {
92 + CopycatServer server = createCopycatServer(members.get(i));
93 + server.open().thenRun(latch::countDown);
94 + servers.add(server);
95 + }
96 +
97 + Uninterruptibles.awaitUninterruptibly(latch);
98 +
99 + return servers;
100 + }
101 +
102 + /**
103 + * Creates a Copycat server.
104 + */
105 + protected CopycatServer createCopycatServer(Address address) {
106 + ResourceRegistry resourceRegistry = new ResourceRegistry();
107 + resourceRegistry.register(resourceType());
108 + CopycatServer server = CopycatServer.builder(address, members)
109 + .withTransport(new LocalTransport(registry))
110 + .withStorage(Storage.builder()
111 + .withStorageLevel(StorageLevel.DISK)
112 + .withDirectory(TEST_DIR + "/" + address.port())
113 + .withSerializer(serializer.clone())
114 + .build())
115 + .withStateMachine(() -> new ResourceManagerState(resourceRegistry))
116 + .withSerializer(serializer.clone())
117 + .withHeartbeatInterval(Duration.ofMillis(25))
118 + .withElectionTimeout(Duration.ofMillis(50))
119 + .withSessionTimeout(Duration.ofMillis(100))
120 + .build();
121 + copycatServers.add(server);
122 + return server;
123 + }
124 +
125 + @Before
126 + @After
127 + public void clearTests() throws Exception {
128 + registry = new LocalServerRegistry();
129 + members = new ArrayList<>();
130 + port = 5000;
131 +
132 + CompletableFuture<Void> closeClients =
133 + CompletableFuture.allOf(atomixClients.stream()
134 + .map(Atomix::close)
135 + .toArray(CompletableFuture[]::new));
136 +
137 + closeClients.thenCompose(v -> CompletableFuture.allOf(copycatServers.stream()
138 + .map(CopycatServer::close)
139 + .toArray(CompletableFuture[]::new))).join();
140 +
141 + deleteDirectory(TEST_DIR);
142 +
143 + atomixClients = new ArrayList<>();
144 +
145 + copycatServers = new ArrayList<>();
146 + }
147 +
148 + /**
149 + * Deletes a directory recursively.
150 + */
151 + private void deleteDirectory(File directory) throws IOException {
152 + if (directory.exists()) {
153 + File[] files = directory.listFiles();
154 + if (files != null) {
155 + for (File file : files) {
156 + if (file.isDirectory()) {
157 + deleteDirectory(file);
158 + } else {
159 + Files.delete(file.toPath());
160 + }
161 + }
162 + }
163 + Files.delete(directory.toPath());
164 + }
165 + }
166 +
167 + /**
168 + * Creates a Atomix client.
169 + */
170 + protected Atomix createAtomixClient() {
171 + CountDownLatch latch = new CountDownLatch(1);
172 + Atomix client = AtomixClient.builder(members)
173 + .withTransport(new LocalTransport(registry))
174 + .withSerializer(serializer.clone())
175 + .withResourceResolver(r -> r.register(resourceType()))
176 + .build();
177 + client.open().thenRun(latch::countDown);
178 + atomixClients.add(client);
179 + Uninterruptibles.awaitUninterruptibly(latch);
180 + return client;
181 + }
182 +}