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 4366 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.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.Collection;
25 +import java.util.ConcurrentModificationException;
26 +import java.util.Map.Entry;
27 +import java.util.Set;
28 +import java.util.concurrent.CompletableFuture;
29 +import java.util.concurrent.atomic.AtomicReference;
30 +import java.util.function.BiFunction;
31 +import java.util.function.Predicate;
32 +
33 +import org.onlab.util.Match;
34 +import org.onosproject.store.service.AsyncConsistentMap;
35 +import org.onosproject.store.service.MapEvent;
36 +import org.onosproject.store.service.MapEventListener;
37 +import org.onosproject.store.service.Versioned;
38 +
39 +import com.google.common.collect.Sets;
40 +
41 +/**
42 + * Distributed resource providing the {@link AsyncConsistentMap} primitive.
43 + */
44 +@ResourceTypeInfo(id = -151, stateMachine = AtomixConsistentMapState.class)
45 +public class AtomixConsistentMap extends Resource<AtomixConsistentMap, Resource.Options>
46 + implements AsyncConsistentMap<String, byte[]> {
47 +
48 + private final Set<MapEventListener<String, byte[]>> mapEventListeners = Sets.newCopyOnWriteArraySet();
49 +
50 + private static final String CHANGE_SUBJECT = "change";
51 +
52 + public AtomixConsistentMap(CopycatClient client, Resource.Options options) {
53 + super(client, options);
54 + }
55 +
56 + @Override
57 + public String name() {
58 + return null;
59 + }
60 +
61 + @Override
62 + public CompletableFuture<AtomixConsistentMap> open() {
63 + return super.open().thenApply(result -> {
64 + client.session().onEvent(CHANGE_SUBJECT, this::handleEvent);
65 + return result;
66 + });
67 + }
68 +
69 + private void handleEvent(MapEvent<String, byte[]> event) {
70 + mapEventListeners.forEach(listener -> listener.event(event));
71 + }
72 +
73 + @Override
74 + public AtomixConsistentMap with(Consistency consistency) {
75 + super.with(consistency);
76 + return this;
77 + }
78 +
79 + @Override
80 + public CompletableFuture<Boolean> isEmpty() {
81 + return submit(new AtomixConsistentMapCommands.IsEmpty());
82 + }
83 +
84 + @Override
85 + public CompletableFuture<Integer> size() {
86 + return submit(new AtomixConsistentMapCommands.Size());
87 + }
88 +
89 + @Override
90 + public CompletableFuture<Boolean> containsKey(String key) {
91 + return submit(new AtomixConsistentMapCommands.ContainsKey(key));
92 + }
93 +
94 + @Override
95 + public CompletableFuture<Boolean> containsValue(byte[] value) {
96 + return submit(new AtomixConsistentMapCommands.ContainsValue(value));
97 + }
98 +
99 + @Override
100 + public CompletableFuture<Versioned<byte[]>> get(String key) {
101 + return submit(new AtomixConsistentMapCommands.Get(key));
102 + }
103 +
104 + @Override
105 + public CompletableFuture<Set<String>> keySet() {
106 + return submit(new AtomixConsistentMapCommands.KeySet());
107 + }
108 +
109 + @Override
110 + public CompletableFuture<Collection<Versioned<byte[]>>> values() {
111 + return submit(new AtomixConsistentMapCommands.Values());
112 + }
113 +
114 + @Override
115 + public CompletableFuture<Set<Entry<String, Versioned<byte[]>>>> entrySet() {
116 + return submit(new AtomixConsistentMapCommands.EntrySet());
117 + }
118 +
119 + @Override
120 + @SuppressWarnings("unchecked")
121 + public CompletableFuture<Versioned<byte[]>> put(String key, byte[] value) {
122 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key, value, Match.ANY, Match.ANY))
123 + .whenComplete((r, e) -> throwIfLocked(r.status()))
124 + .thenApply(v -> v.oldValue());
125 + }
126 +
127 + @Override
128 + @SuppressWarnings("unchecked")
129 + public CompletableFuture<Versioned<byte[]>> putAndGet(String key, byte[] value) {
130 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key, value, Match.ANY, Match.ANY))
131 + .whenComplete((r, e) -> throwIfLocked(r.status()))
132 + .thenApply(v -> v.newValue());
133 + }
134 +
135 + @Override
136 + @SuppressWarnings("unchecked")
137 + public CompletableFuture<Versioned<byte[]>> putIfAbsent(String key, byte[] value) {
138 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key, value, Match.NULL, Match.ANY))
139 + .whenComplete((r, e) -> throwIfLocked(r.status()))
140 + .thenApply(v -> v.oldValue());
141 + }
142 + @Override
143 + @SuppressWarnings("unchecked")
144 + public CompletableFuture<Versioned<byte[]>> remove(String key) {
145 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key, null, Match.ANY, Match.ANY))
146 + .whenComplete((r, e) -> throwIfLocked(r.status()))
147 + .thenApply(v -> v.oldValue());
148 + }
149 +
150 + @Override
151 + @SuppressWarnings("unchecked")
152 + public CompletableFuture<Boolean> remove(String key, byte[] value) {
153 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key, null, Match.ifValue(value), Match.ANY))
154 + .whenComplete((r, e) -> throwIfLocked(r.status()))
155 + .thenApply(v -> v.updated());
156 + }
157 +
158 + @Override
159 + @SuppressWarnings("unchecked")
160 + public CompletableFuture<Boolean> remove(String key, long version) {
161 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key, null, Match.ANY, Match.ifValue(version)))
162 + .whenComplete((r, e) -> throwIfLocked(r.status()))
163 + .thenApply(v -> v.updated());
164 + }
165 +
166 + @Override
167 + @SuppressWarnings("unchecked")
168 + public CompletableFuture<Versioned<byte[]>> replace(String key, byte[] value) {
169 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key, value, Match.NOT_NULL, Match.ANY))
170 + .whenComplete((r, e) -> throwIfLocked(r.status()))
171 + .thenApply(v -> v.oldValue());
172 + }
173 +
174 + @Override
175 + @SuppressWarnings("unchecked")
176 + public CompletableFuture<Boolean> replace(String key, byte[] oldValue, byte[] newValue) {
177 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key,
178 + newValue,
179 + Match.ifValue(oldValue),
180 + Match.ANY))
181 + .whenComplete((r, e) -> throwIfLocked(r.status()))
182 + .thenApply(v -> v.updated());
183 + }
184 +
185 + @Override
186 + @SuppressWarnings("unchecked")
187 + public CompletableFuture<Boolean> replace(String key, long oldVersion, byte[] newValue) {
188 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key,
189 + newValue,
190 + Match.ANY,
191 + Match.ifValue(oldVersion)))
192 + .whenComplete((r, e) -> throwIfLocked(r.status()))
193 + .thenApply(v -> v.updated());
194 + }
195 +
196 + @Override
197 + public CompletableFuture<Void> clear() {
198 + return submit(new AtomixConsistentMapCommands.Clear())
199 + .whenComplete((r, e) -> throwIfLocked(r))
200 + .thenApply(v -> null);
201 + }
202 +
203 + @Override
204 + @SuppressWarnings("unchecked")
205 + public CompletableFuture<Versioned<byte[]>> computeIf(String key,
206 + Predicate<? super byte[]> condition,
207 + BiFunction<? super String, ? super byte[], ? extends byte[]> remappingFunction) {
208 + return get(key).thenCompose(r1 -> {
209 + byte[] existingValue = r1 == null ? null : r1.value();
210 + // if the condition evaluates to false, return existing value.
211 + if (!condition.test(existingValue)) {
212 + return CompletableFuture.completedFuture(r1);
213 + }
214 +
215 + AtomicReference<byte[]> computedValue = new AtomicReference<>();
216 + // if remappingFunction throws an exception, return the exception.
217 + try {
218 + computedValue.set(remappingFunction.apply(key, existingValue));
219 + } catch (Exception e) {
220 + CompletableFuture<Versioned<byte[]>> future = new CompletableFuture<>();
221 + future.completeExceptionally(e);
222 + return future;
223 + }
224 + if (computedValue.get() == null && r1 == null) {
225 + return CompletableFuture.completedFuture(null);
226 + }
227 + Match<byte[]> valueMatch = r1 == null ? Match.NULL : Match.ANY;
228 + Match<Long> versionMatch = r1 == null ? Match.ANY : Match.ifValue(r1.version());
229 + return submit(new AtomixConsistentMapCommands.UpdateAndGet(key,
230 + computedValue.get(),
231 + valueMatch,
232 + versionMatch))
233 + .whenComplete((r, e) -> throwIfLocked(r.status()))
234 + .thenApply(v -> v.newValue());
235 + });
236 + }
237 +
238 + public CompletableFuture<PrepareResult> prepare(TransactionalMapUpdate<String, byte[]> update) {
239 + return submit(new AtomixConsistentMapCommands.TransactionPrepare(update));
240 + }
241 +
242 + public CompletableFuture<CommitResult> commit(TransactionId transactionId) {
243 + return submit(new AtomixConsistentMapCommands.TransactionCommit(transactionId));
244 + }
245 +
246 + public CompletableFuture<RollbackResult> rollback(TransactionId transactionId) {
247 + return submit(new AtomixConsistentMapCommands.TransactionRollback(transactionId));
248 + }
249 +
250 + @Override
251 + public synchronized CompletableFuture<Void> addListener(MapEventListener<String, byte[]> listener) {
252 + if (!mapEventListeners.isEmpty()) {
253 + if (mapEventListeners.add(listener)) {
254 + return CompletableFuture.completedFuture(new ChangeListener(listener)).thenApply(v -> null);
255 + } else {
256 + return CompletableFuture.completedFuture(null);
257 + }
258 + }
259 + mapEventListeners.add(listener);
260 + return submit(new AtomixConsistentMapCommands.Listen()).thenApply(v -> null);
261 + }
262 +
263 + @Override
264 + public synchronized CompletableFuture<Void> removeListener(MapEventListener<String, byte[]> listener) {
265 + if (mapEventListeners.remove(listener) && mapEventListeners.isEmpty()) {
266 + return submit(new AtomixConsistentMapCommands.Unlisten()).thenApply(v -> null);
267 + }
268 + return CompletableFuture.completedFuture(null);
269 + }
270 +
271 + private void throwIfLocked(MapEntryUpdateResult.Status status) {
272 + if (status == MapEntryUpdateResult.Status.WRITE_LOCK) {
273 + throw new ConcurrentModificationException("Cannot update map: Another transaction in progress");
274 + }
275 + }
276 +
277 + /**
278 + * Change listener context.
279 + */
280 + private final class ChangeListener implements Listener<MapEvent<String, byte[]>> {
281 + private final MapEventListener<String, byte[]> listener;
282 +
283 + private ChangeListener(MapEventListener<String, byte[]> listener) {
284 + this.listener = listener;
285 + }
286 +
287 + @Override
288 + public void accept(MapEvent<String, byte[]> event) {
289 + listener.event(event);
290 + }
291 +
292 + @Override
293 + public void close() {
294 + synchronized (AtomixConsistentMap.this) {
295 + mapEventListeners.remove(listener);
296 + if (mapEventListeners.isEmpty()) {
297 + submit(new AtomixConsistentMapCommands.Unlisten());
298 + }
299 + }
300 + }
301 + }
302 +}
...\ 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.buffer.BufferInput;
19 +import io.atomix.catalyst.buffer.BufferOutput;
20 +import io.atomix.catalyst.serializer.CatalystSerializable;
21 +import io.atomix.catalyst.serializer.Serializer;
22 +import io.atomix.catalyst.util.Assert;
23 +import io.atomix.copycat.client.Command;
24 +import io.atomix.copycat.client.Query;
25 +
26 +import java.util.Collection;
27 +import java.util.Map;
28 +import java.util.Set;
29 +
30 +import org.onlab.util.Match;
31 +import org.onosproject.store.service.Versioned;
32 +
33 +import com.google.common.base.MoreObjects;
34 +
35 +/**
36 + * {@link AtomixConsistentMap} resource state machine operations.
37 + */
38 +public final class AtomixConsistentMapCommands {
39 +
40 + private AtomixConsistentMapCommands() {
41 + }
42 +
43 + /**
44 + * Abstract map command.
45 + */
46 + @SuppressWarnings("serial")
47 + public abstract static class MapCommand<V> implements Command<V>, CatalystSerializable {
48 +
49 + @Override
50 + public ConsistencyLevel consistency() {
51 + return ConsistencyLevel.LINEARIZABLE;
52 + }
53 +
54 + @Override
55 + public String toString() {
56 + return MoreObjects.toStringHelper(getClass())
57 + .toString();
58 + }
59 +
60 + @Override
61 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
62 + }
63 +
64 + @Override
65 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
66 + }
67 + }
68 +
69 + /**
70 + * Abstract map query.
71 + */
72 + @SuppressWarnings("serial")
73 + public abstract static class MapQuery<V> implements Query<V>, CatalystSerializable {
74 +
75 + @Override
76 + public ConsistencyLevel consistency() {
77 + return ConsistencyLevel.BOUNDED_LINEARIZABLE;
78 + }
79 +
80 + @Override
81 + public String toString() {
82 + return MoreObjects.toStringHelper(getClass())
83 + .toString();
84 + }
85 +
86 + @Override
87 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
88 + }
89 +
90 + @Override
91 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
92 + }
93 + }
94 +
95 + /**
96 + * Abstract key-based query.
97 + */
98 + @SuppressWarnings("serial")
99 + public abstract static class KeyQuery<V> extends MapQuery<V> {
100 + protected String key;
101 +
102 + public KeyQuery() {
103 + }
104 +
105 + public KeyQuery(String key) {
106 + this.key = Assert.notNull(key, "key");
107 + }
108 +
109 + /**
110 + * Returns the key.
111 + * @return key
112 + */
113 + public String key() {
114 + return key;
115 + }
116 +
117 + @Override
118 + public String toString() {
119 + return MoreObjects.toStringHelper(getClass())
120 + .add("key", key)
121 + .toString();
122 + }
123 +
124 + @Override
125 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
126 + super.writeObject(buffer, serializer);
127 + serializer.writeObject(key, buffer);
128 + }
129 +
130 + @Override
131 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
132 + super.readObject(buffer, serializer);
133 + key = serializer.readObject(buffer);
134 + }
135 + }
136 +
137 + /**
138 + * Abstract key-based query.
139 + */
140 + @SuppressWarnings("serial")
141 + public abstract static class ValueQuery<V> extends MapQuery<V> {
142 + protected byte[] value;
143 +
144 + public ValueQuery() {
145 + }
146 +
147 + public ValueQuery(byte[] value) {
148 + this.value = Assert.notNull(value, "value");
149 + }
150 +
151 + /**
152 + * Returns the key.
153 + * @return key
154 + */
155 + public byte[] value() {
156 + return value;
157 + }
158 +
159 + @Override
160 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
161 + super.writeObject(buffer, serializer);
162 + serializer.writeObject(value, buffer);
163 + }
164 +
165 + @Override
166 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
167 + super.readObject(buffer, serializer);
168 + value = serializer.readObject(buffer);
169 + }
170 + }
171 +
172 + /**
173 + * Contains key command.
174 + */
175 + @SuppressWarnings("serial")
176 + public static class ContainsKey extends KeyQuery<Boolean> {
177 + public ContainsKey() {
178 + }
179 +
180 + public ContainsKey(String key) {
181 + super(key);
182 + }
183 + }
184 +
185 + /**
186 + * Contains key command.
187 + */
188 + @SuppressWarnings("serial")
189 + public static class ContainsValue extends ValueQuery<Boolean> {
190 + public ContainsValue() {
191 + }
192 +
193 + public ContainsValue(byte[] value) {
194 + super(value);
195 + }
196 +
197 + @Override
198 + public String toString() {
199 + return MoreObjects.toStringHelper(getClass())
200 + .add("value", value)
201 + .toString();
202 + }
203 + }
204 +
205 + /**
206 + * Map prepare command.
207 + */
208 + @SuppressWarnings("serial")
209 + public static class TransactionPrepare extends MapCommand<PrepareResult> {
210 + private TransactionalMapUpdate<String, byte[]> update;
211 +
212 + public TransactionPrepare() {
213 + }
214 +
215 + public TransactionPrepare(TransactionalMapUpdate<String, byte[]> update) {
216 + this.update = update;
217 + }
218 +
219 + public TransactionalMapUpdate<String, byte[]> transactionUpdate() {
220 + return update;
221 + }
222 +
223 + @Override
224 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
225 + super.writeObject(buffer, serializer);
226 + serializer.writeObject(update, buffer);
227 + }
228 +
229 + @Override
230 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
231 + super.readObject(buffer, serializer);
232 + update = serializer.readObject(buffer);
233 + }
234 +
235 + @Override
236 + public String toString() {
237 + return MoreObjects.toStringHelper(getClass())
238 + .add("update", update)
239 + .toString();
240 + }
241 + }
242 +
243 + /**
244 + * Map transaction commit command.
245 + */
246 + @SuppressWarnings("serial")
247 + public static class TransactionCommit extends MapCommand<CommitResult> {
248 + private TransactionId transactionId;
249 +
250 + public TransactionCommit() {
251 + }
252 +
253 + public TransactionCommit(TransactionId transactionId) {
254 + this.transactionId = transactionId;
255 + }
256 +
257 + /**
258 + * Returns the transaction identifier.
259 + * @return transaction id
260 + */
261 + public TransactionId transactionId() {
262 + return transactionId;
263 + }
264 +
265 + @Override
266 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
267 + super.writeObject(buffer, serializer);
268 + serializer.writeObject(transactionId, buffer);
269 + }
270 +
271 + @Override
272 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
273 + super.readObject(buffer, serializer);
274 + transactionId = serializer.readObject(buffer);
275 + }
276 +
277 + @Override
278 + public String toString() {
279 + return MoreObjects.toStringHelper(getClass())
280 + .add("transactionId", transactionId)
281 + .toString();
282 + }
283 + }
284 +
285 + /**
286 + * Map transaction rollback command.
287 + */
288 + @SuppressWarnings("serial")
289 + public static class TransactionRollback extends MapCommand<RollbackResult> {
290 + private TransactionId transactionId;
291 +
292 + public TransactionRollback() {
293 + }
294 +
295 + public TransactionRollback(TransactionId transactionId) {
296 + this.transactionId = transactionId;
297 + }
298 +
299 + /**
300 + * Returns the transaction identifier.
301 + * @return transaction id
302 + */
303 + public TransactionId transactionId() {
304 + return transactionId;
305 + }
306 +
307 + @Override
308 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
309 + super.writeObject(buffer, serializer);
310 + serializer.writeObject(transactionId, buffer);
311 + }
312 +
313 + @Override
314 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
315 + super.readObject(buffer, serializer);
316 + transactionId = serializer.readObject(buffer);
317 + }
318 +
319 + @Override
320 + public String toString() {
321 + return MoreObjects.toStringHelper(getClass())
322 + .add("transactionId", transactionId)
323 + .toString();
324 + }
325 + }
326 +
327 + /**
328 + * Map update command.
329 + */
330 + @SuppressWarnings("serial")
331 + public static class UpdateAndGet extends MapCommand<MapEntryUpdateResult<String, byte[]>> {
332 + private String key;
333 + private byte[] value;
334 + private Match<byte[]> valueMatch;
335 + private Match<Long> versionMatch;
336 +
337 + public UpdateAndGet() {
338 + }
339 +
340 + public UpdateAndGet(String key,
341 + byte[] value,
342 + Match<byte[]> valueMatch,
343 + Match<Long> versionMatch) {
344 + this.key = key;
345 + this.value = value;
346 + this.valueMatch = valueMatch;
347 + this.versionMatch = versionMatch;
348 + }
349 +
350 + /**
351 + * Returns the key.
352 + * @return key
353 + */
354 + public String key() {
355 + return this.key;
356 + }
357 +
358 + /**
359 + * Returns the value.
360 + * @return value
361 + */
362 + public byte[] value() {
363 + return this.value;
364 + }
365 +
366 + /**
367 + * Returns the value match.
368 + * @return value match
369 + */
370 + public Match<byte[]> valueMatch() {
371 + return this.valueMatch;
372 + }
373 +
374 + /**
375 + * Returns the version match.
376 + * @return version match
377 + */
378 + public Match<Long> versionMatch() {
379 + return this.versionMatch;
380 + }
381 +
382 + @Override
383 + public CompactionMode compaction() {
384 + return value == null ? CompactionMode.FULL : CompactionMode.QUORUM;
385 + }
386 +
387 + @Override
388 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
389 + super.writeObject(buffer, serializer);
390 + serializer.writeObject(key, buffer);
391 + serializer.writeObject(value, buffer);
392 + serializer.writeObject(valueMatch, buffer);
393 + serializer.writeObject(versionMatch, buffer);
394 + }
395 +
396 + @Override
397 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
398 + super.readObject(buffer, serializer);
399 + key = serializer.readObject(buffer);
400 + value = serializer.readObject(buffer);
401 + valueMatch = serializer.readObject(buffer);
402 + versionMatch = serializer.readObject(buffer);
403 + }
404 +
405 + @Override
406 + public String toString() {
407 + return MoreObjects.toStringHelper(getClass())
408 + .add("key", key)
409 + .add("value", value)
410 + .add("valueMatch", valueMatch)
411 + .add("versionMatch", versionMatch)
412 + .toString();
413 + }
414 + }
415 +
416 + /**
417 + * Get query.
418 + */
419 + @SuppressWarnings("serial")
420 + public static class Get extends KeyQuery<Versioned<byte[]>> {
421 + public Get() {
422 + }
423 +
424 + public Get(String key) {
425 + super(key);
426 + }
427 + }
428 +
429 + /**
430 + * Is empty query.
431 + */
432 + @SuppressWarnings("serial")
433 + public static class IsEmpty extends MapQuery<Boolean> {
434 + }
435 +
436 + /**
437 + * KeySet query.
438 + */
439 + @SuppressWarnings("serial")
440 + public static class KeySet extends MapQuery<Set<String>> {
441 + }
442 +
443 + /**
444 + * KeySet query.
445 + */
446 + @SuppressWarnings("serial")
447 + public static class Values extends MapQuery<Collection<Versioned<byte[]>>> {
448 + }
449 +
450 + /**
451 + * KeySet query.
452 + */
453 + @SuppressWarnings("serial")
454 + public static class EntrySet extends MapQuery<Set<Map.Entry<String, Versioned<byte[]>>>> {
455 + }
456 +
457 + /**
458 + * Size query.
459 + */
460 + @SuppressWarnings("serial")
461 + public static class Size extends MapQuery<Integer> {
462 + }
463 +
464 + /**
465 + * Clear command.
466 + */
467 + @SuppressWarnings("serial")
468 + public static class Clear extends MapCommand<MapEntryUpdateResult.Status> {
469 +
470 + @Override
471 + public CompactionMode compaction() {
472 + return CompactionMode.FULL;
473 + }
474 + }
475 +
476 + /**
477 + * Change listen.
478 + */
479 + @SuppressWarnings("serial")
480 + public static class Listen implements Command<Void>, CatalystSerializable {
481 + @Override
482 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
483 + }
484 +
485 + @Override
486 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
487 + }
488 +
489 + @Override
490 + public String toString() {
491 + return MoreObjects.toStringHelper(getClass())
492 + .toString();
493 + }
494 + }
495 +
496 + /**
497 + * Change unlisten.
498 + */
499 + @SuppressWarnings("serial")
500 + public static class Unlisten implements Command<Void>, CatalystSerializable {
501 + @Override
502 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
503 + }
504 +
505 + @Override
506 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
507 + }
508 +
509 + @Override
510 + public String toString() {
511 + return MoreObjects.toStringHelper(getClass())
512 + .toString();
513 + }
514 + }
515 +}
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.onosproject.store.service.MapEvent.Type.INSERT;
19 +import static org.onosproject.store.service.MapEvent.Type.REMOVE;
20 +import static org.onosproject.store.service.MapEvent.Type.UPDATE;
21 +import io.atomix.copycat.client.session.Session;
22 +import io.atomix.copycat.server.Commit;
23 +import io.atomix.copycat.server.Snapshottable;
24 +import io.atomix.copycat.server.StateMachineExecutor;
25 +import io.atomix.copycat.server.session.SessionListener;
26 +import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
27 +import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
28 +import io.atomix.resource.ResourceStateMachine;
29 +
30 +import java.util.Collection;
31 +import java.util.HashMap;
32 +import java.util.Iterator;
33 +import java.util.Map;
34 +import java.util.Set;
35 +import java.util.concurrent.atomic.AtomicLong;
36 +import java.util.stream.Collectors;
37 +
38 +import org.onlab.util.CountDownCompleter;
39 +import org.onlab.util.Match;
40 +import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionPrepare;
41 +import org.onosproject.store.service.MapEvent;
42 +import org.onosproject.store.service.Versioned;
43 +
44 +import com.google.common.collect.Maps;
45 +import com.google.common.collect.Sets;
46 +
47 +import static com.google.common.base.Preconditions.checkState;
48 +
49 +/**
50 + * State Machine for {@link AtomixConsistentMap} resource.
51 + */
52 +public class AtomixConsistentMapState extends ResourceStateMachine implements
53 + SessionListener, Snapshottable {
54 + private final Map<Long, Commit<? extends AtomixConsistentMapCommands.Listen>> listeners = new HashMap<>();
55 + private final Map<String, MapEntryValue> mapEntries = new HashMap<>();
56 + private final Set<String> preparedKeys = Sets.newHashSet();
57 + private final Map<TransactionId, Commit<? extends TransactionPrepare>> pendingTransactions = Maps
58 + .newHashMap();
59 + private AtomicLong versionCounter = new AtomicLong(0);
60 +
61 + @Override
62 + public void snapshot(SnapshotWriter writer) {
63 + writer.writeLong(versionCounter.get());
64 + }
65 +
66 + @Override
67 + public void install(SnapshotReader reader) {
68 + versionCounter = new AtomicLong(reader.readLong());
69 + }
70 +
71 + @Override
72 + protected void configure(StateMachineExecutor executor) {
73 + // Listeners
74 + executor.register(AtomixConsistentMapCommands.Listen.class,
75 + this::listen);
76 + executor.register(AtomixConsistentMapCommands.Unlisten.class,
77 + this::unlisten);
78 + // Queries
79 + executor.register(AtomixConsistentMapCommands.ContainsKey.class,
80 + this::containsKey);
81 + executor.register(AtomixConsistentMapCommands.ContainsValue.class,
82 + this::containsValue);
83 + executor.register(AtomixConsistentMapCommands.EntrySet.class,
84 + this::entrySet);
85 + executor.register(AtomixConsistentMapCommands.Get.class, this::get);
86 + executor.register(AtomixConsistentMapCommands.IsEmpty.class,
87 + this::isEmpty);
88 + executor.register(AtomixConsistentMapCommands.KeySet.class,
89 + this::keySet);
90 + executor.register(AtomixConsistentMapCommands.Size.class, this::size);
91 + executor.register(AtomixConsistentMapCommands.Values.class,
92 + this::values);
93 + // Commands
94 + executor.register(AtomixConsistentMapCommands.UpdateAndGet.class,
95 + this::updateAndGet);
96 + executor.register(AtomixConsistentMapCommands.Clear.class, this::clear);
97 + executor.register(AtomixConsistentMapCommands.TransactionPrepare.class,
98 + this::prepare);
99 + executor.register(AtomixConsistentMapCommands.TransactionCommit.class,
100 + this::commit);
101 + executor.register(
102 + AtomixConsistentMapCommands.TransactionRollback.class,
103 + this::rollback);
104 + }
105 +
106 + @Override
107 + public void delete() {
108 + // Delete Listeners
109 + listeners.values().forEach(Commit::close);
110 + listeners.clear();
111 +
112 + // Delete Map entries
113 + mapEntries.values().forEach(MapEntryValue::discard);
114 + mapEntries.clear();
115 + }
116 +
117 + /**
118 + * Handles a contains key commit.
119 + *
120 + * @param commit
121 + * containsKey commit
122 + * @return {@code true} if map contains key
123 + */
124 + protected boolean containsKey(
125 + Commit<? extends AtomixConsistentMapCommands.ContainsKey> commit) {
126 + try {
127 + return toVersioned(mapEntries.get(commit.operation().key())) != null;
128 + } finally {
129 + commit.close();
130 + }
131 + }
132 +
133 + /**
134 + * Handles a contains value commit.
135 + *
136 + * @param commit
137 + * containsValue commit
138 + * @return {@code true} if map contains value
139 + */
140 + protected boolean containsValue(
141 + Commit<? extends AtomixConsistentMapCommands.ContainsValue> commit) {
142 + try {
143 + Match<byte[]> valueMatch = Match
144 + .ifValue(commit.operation().value());
145 + return mapEntries.values().stream()
146 + .anyMatch(value -> valueMatch.matches(value.value()));
147 + } finally {
148 + commit.close();
149 + }
150 + }
151 +
152 + /**
153 + * Handles a get commit.
154 + *
155 + * @param commit
156 + * get commit
157 + * @return value mapped to key
158 + */
159 + protected Versioned<byte[]> get(
160 + Commit<? extends AtomixConsistentMapCommands.Get> commit) {
161 + try {
162 + return toVersioned(mapEntries.get(commit.operation().key()));
163 + } finally {
164 + commit.close();
165 + }
166 + }
167 +
168 + /**
169 + * Handles a count commit.
170 + *
171 + * @param commit
172 + * size commit
173 + * @return number of entries in map
174 + */
175 + protected int size(Commit<? extends AtomixConsistentMapCommands.Size> commit) {
176 + try {
177 + return mapEntries.size();
178 + } finally {
179 + commit.close();
180 + }
181 + }
182 +
183 + /**
184 + * Handles an is empty commit.
185 + *
186 + * @param commit
187 + * isEmpty commit
188 + * @return {@code true} if map is empty
189 + */
190 + protected boolean isEmpty(
191 + Commit<? extends AtomixConsistentMapCommands.IsEmpty> commit) {
192 + try {
193 + return mapEntries.isEmpty();
194 + } finally {
195 + commit.close();
196 + }
197 + }
198 +
199 + /**
200 + * Handles a keySet commit.
201 + *
202 + * @param commit
203 + * keySet commit
204 + * @return set of keys in map
205 + */
206 + protected Set<String> keySet(
207 + Commit<? extends AtomixConsistentMapCommands.KeySet> commit) {
208 + try {
209 + return mapEntries.keySet();
210 + } finally {
211 + commit.close();
212 + }
213 + }
214 +
215 + /**
216 + * Handles a values commit.
217 + *
218 + * @param commit
219 + * values commit
220 + * @return collection of values in map
221 + */
222 + protected Collection<Versioned<byte[]>> values(
223 + Commit<? extends AtomixConsistentMapCommands.Values> commit) {
224 + try {
225 + return mapEntries.values().stream().map(this::toVersioned)
226 + .collect(Collectors.toList());
227 + } finally {
228 + commit.close();
229 + }
230 + }
231 +
232 + /**
233 + * Handles a entry set commit.
234 + *
235 + * @param commit
236 + * entrySet commit
237 + * @return set of map entries
238 + */
239 + protected Set<Map.Entry<String, Versioned<byte[]>>> entrySet(
240 + Commit<? extends AtomixConsistentMapCommands.EntrySet> commit) {
241 + try {
242 + return mapEntries
243 + .entrySet()
244 + .stream()
245 + .map(e -> Maps.immutableEntry(e.getKey(),
246 + toVersioned(e.getValue())))
247 + .collect(Collectors.toSet());
248 + } finally {
249 + commit.close();
250 + }
251 + }
252 +
253 + /**
254 + * Handles a update and get commit.
255 + *
256 + * @param commit
257 + * updateAndGet commit
258 + * @return update result
259 + */
260 + protected MapEntryUpdateResult<String, byte[]> updateAndGet(
261 + Commit<? extends AtomixConsistentMapCommands.UpdateAndGet> commit) {
262 + MapEntryUpdateResult.Status updateStatus = validate(commit.operation());
263 + String key = commit.operation().key();
264 + MapEntryValue oldCommitValue = mapEntries.get(commit.operation().key());
265 + Versioned<byte[]> oldMapValue = toVersioned(oldCommitValue);
266 +
267 + if (updateStatus != MapEntryUpdateResult.Status.OK) {
268 + commit.close();
269 + return new MapEntryUpdateResult<>(updateStatus, "", key,
270 + oldMapValue, oldMapValue);
271 + }
272 +
273 + byte[] newValue = commit.operation().value();
274 + long newVersion = versionCounter.incrementAndGet();
275 + Versioned<byte[]> newMapValue = newValue == null ? null
276 + : new Versioned<>(newValue, newVersion);
277 +
278 + MapEvent.Type updateType = newValue == null ? REMOVE
279 + : oldCommitValue == null ? INSERT : UPDATE;
280 + if (updateType == REMOVE || updateType == UPDATE) {
281 + mapEntries.remove(key);
282 + oldCommitValue.discard();
283 + }
284 + if (updateType == INSERT || updateType == UPDATE) {
285 + mapEntries.put(key, new NonTransactionalCommit(newVersion, commit));
286 + }
287 + notify(new MapEvent<>("", key, newMapValue, oldMapValue));
288 + return new MapEntryUpdateResult<>(updateStatus, "", key, oldMapValue,
289 + newMapValue);
290 + }
291 +
292 + /**
293 + * Handles a clear commit.
294 + *
295 + * @param commit
296 + * clear commit
297 + * @return clear result
298 + */
299 + protected MapEntryUpdateResult.Status clear(
300 + Commit<? extends AtomixConsistentMapCommands.Clear> commit) {
301 + try {
302 + Iterator<Map.Entry<String, MapEntryValue>> iterator = mapEntries
303 + .entrySet().iterator();
304 + while (iterator.hasNext()) {
305 + Map.Entry<String, MapEntryValue> entry = iterator.next();
306 + String key = entry.getKey();
307 + MapEntryValue value = entry.getValue();
308 + Versioned<byte[]> removedValue = new Versioned<>(value.value(),
309 + value.version());
310 + notify(new MapEvent<>("", key, null, removedValue));
311 + value.discard();
312 + iterator.remove();
313 + }
314 + return MapEntryUpdateResult.Status.OK;
315 + } finally {
316 + commit.close();
317 + }
318 + }
319 +
320 + /**
321 + * Handles a listen commit.
322 + *
323 + * @param commit
324 + * listen commit
325 + */
326 + protected void listen(
327 + Commit<? extends AtomixConsistentMapCommands.Listen> commit) {
328 + Long sessionId = commit.session().id();
329 + listeners.put(sessionId, commit);
330 + commit.session()
331 + .onStateChange(
332 + state -> {
333 + if (state == Session.State.CLOSED
334 + || state == Session.State.EXPIRED) {
335 + Commit<? extends AtomixConsistentMapCommands.Listen> listener = listeners
336 + .remove(sessionId);
337 + if (listener != null) {
338 + listener.close();
339 + }
340 + }
341 + });
342 + }
343 +
344 + /**
345 + * Handles an unlisten commit.
346 + *
347 + * @param commit
348 + * unlisten commit
349 + */
350 + protected void unlisten(
351 + Commit<? extends AtomixConsistentMapCommands.Unlisten> commit) {
352 + try {
353 + Commit<? extends AtomixConsistentMapCommands.Listen> listener = listeners
354 + .remove(commit.session());
355 + if (listener != null) {
356 + listener.close();
357 + }
358 + } finally {
359 + commit.close();
360 + }
361 + }
362 +
363 + /**
364 + * Triggers a change event.
365 + *
366 + * @param value
367 + * map event
368 + */
369 + private void notify(MapEvent<String, byte[]> value) {
370 + listeners.values().forEach(
371 + commit -> commit.session().publish("change", value));
372 + }
373 +
374 + /**
375 + * Handles an prepare commit.
376 + *
377 + * @param commit
378 + * transaction prepare commit
379 + * @return prepare result
380 + */
381 + protected PrepareResult prepare(
382 + Commit<? extends AtomixConsistentMapCommands.TransactionPrepare> commit) {
383 + boolean ok = false;
384 + try {
385 + TransactionalMapUpdate<String, byte[]> transactionUpdate = commit
386 + .operation().transactionUpdate();
387 + for (MapUpdate<String, byte[]> update : transactionUpdate.batch()) {
388 + String key = update.key();
389 + if (preparedKeys.contains(key)) {
390 + return PrepareResult.CONCURRENT_TRANSACTION;
391 + }
392 + MapEntryValue existingValue = mapEntries.get(key);
393 + if (existingValue == null) {
394 + if (update.currentValue() != null) {
395 + return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
396 + }
397 + } else {
398 + if (existingValue.version() != update.currentVersion()) {
399 + return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
400 + }
401 + }
402 + }
403 + // No violations detected. Add to pendingTranctions and mark
404 + // modified keys as
405 + // currently locked to updates.
406 + pendingTransactions.put(transactionUpdate.transactionId(), commit);
407 + transactionUpdate.batch().forEach(u -> preparedKeys.add(u.key()));
408 + ok = true;
409 + return PrepareResult.OK;
410 + } finally {
411 + if (!ok) {
412 + commit.close();
413 + }
414 + }
415 + }
416 +
417 + /**
418 + * Handles an commit commit (ha!).
419 + *
420 + * @param commit transaction commit commit
421 + * @return commit result
422 + */
423 + protected CommitResult commit(
424 + Commit<? extends AtomixConsistentMapCommands.TransactionCommit> commit) {
425 + TransactionId transactionId = commit.operation().transactionId();
426 + try {
427 + Commit<? extends AtomixConsistentMapCommands.TransactionPrepare> prepareCommit = pendingTransactions
428 + .remove(transactionId);
429 + if (prepareCommit == null) {
430 + return CommitResult.UNKNOWN_TRANSACTION_ID;
431 + }
432 + TransactionalMapUpdate<String, byte[]> transactionalUpdate = prepareCommit
433 + .operation().transactionUpdate();
434 + long totalReferencesToCommit = transactionalUpdate
435 + .batch()
436 + .stream()
437 + .filter(update -> update.type() != MapUpdate.Type.REMOVE_IF_VERSION_MATCH)
438 + .count();
439 + CountDownCompleter<Commit<? extends AtomixConsistentMapCommands.TransactionPrepare>> completer =
440 + new CountDownCompleter<>(prepareCommit, totalReferencesToCommit, Commit::close);
441 + for (MapUpdate<String, byte[]> update : transactionalUpdate.batch()) {
442 + String key = update.key();
443 + MapEntryValue previousValue = mapEntries.remove(key);
444 + MapEntryValue newValue = null;
445 + checkState(preparedKeys.remove(key), "key is not prepared");
446 + if (update.type() != MapUpdate.Type.REMOVE_IF_VERSION_MATCH) {
447 + newValue = new TransactionalCommit(key,
448 + versionCounter.incrementAndGet(), completer);
449 + }
450 + mapEntries.put(key, newValue);
451 + // Notify map listeners
452 + notify(new MapEvent<>("", key, toVersioned(newValue),
453 + toVersioned(previousValue)));
454 + }
455 + return CommitResult.OK;
456 + } finally {
457 + commit.close();
458 + }
459 + }
460 +
461 + /**
462 + * Handles an rollback commit (ha!).
463 + *
464 + * @param commit transaction rollback commit
465 + * @return rollback result
466 + */
467 + protected RollbackResult rollback(
468 + Commit<? extends AtomixConsistentMapCommands.TransactionRollback> commit) {
469 + TransactionId transactionId = commit.operation().transactionId();
470 + try {
471 + Commit<? extends AtomixConsistentMapCommands.TransactionPrepare> prepareCommit = pendingTransactions
472 + .remove(transactionId);
473 + if (prepareCommit == null) {
474 + return RollbackResult.UNKNOWN_TRANSACTION_ID;
475 + } else {
476 + prepareCommit.operation().transactionUpdate().batch()
477 + .forEach(u -> preparedKeys.remove(u.key()));
478 + prepareCommit.close();
479 + return RollbackResult.OK;
480 + }
481 + } finally {
482 + commit.close();
483 + }
484 + }
485 +
486 + private MapEntryUpdateResult.Status validate(
487 + AtomixConsistentMapCommands.UpdateAndGet update) {
488 + MapEntryValue existingValue = mapEntries.get(update.key());
489 + if (existingValue == null && update.value() == null) {
490 + return MapEntryUpdateResult.Status.NOOP;
491 + }
492 + if (preparedKeys.contains(update.key())) {
493 + return MapEntryUpdateResult.Status.WRITE_LOCK;
494 + }
495 + byte[] existingRawValue = existingValue == null ? null : existingValue
496 + .value();
497 + Long existingVersion = existingValue == null ? null : existingValue
498 + .version();
499 + return update.valueMatch().matches(existingRawValue)
500 + && update.versionMatch().matches(existingVersion) ? MapEntryUpdateResult.Status.OK
501 + : MapEntryUpdateResult.Status.PRECONDITION_FAILED;
502 + }
503 +
504 + private Versioned<byte[]> toVersioned(MapEntryValue value) {
505 + return value == null ? null : new Versioned<>(value.value(),
506 + value.version());
507 + }
508 +
509 + @Override
510 + public void register(Session session) {
511 + }
512 +
513 + @Override
514 + public void unregister(Session session) {
515 + closeListener(session.id());
516 + }
517 +
518 + @Override
519 + public void expire(Session session) {
520 + closeListener(session.id());
521 + }
522 +
523 + @Override
524 + public void close(Session session) {
525 + closeListener(session.id());
526 + }
527 +
528 + private void closeListener(Long sessionId) {
529 + Commit<? extends AtomixConsistentMapCommands.Listen> commit = listeners
530 + .remove(sessionId);
531 + if (commit != null) {
532 + commit.close();
533 + }
534 + }
535 +
536 + /**
537 + * Interface implemented by map values.
538 + */
539 + private interface MapEntryValue {
540 + /**
541 + * Returns the raw {@code byte[]}.
542 + *
543 + * @return raw value
544 + */
545 + byte[] value();
546 +
547 + /**
548 + * Returns the version of the value.
549 + *
550 + * @return version
551 + */
552 + long version();
553 +
554 + /**
555 + * Discards the value by invoke appropriate clean up actions.
556 + */
557 + void discard();
558 + }
559 +
560 + /**
561 + * A {@code MapEntryValue} that is derived from a non-transactional update
562 + * i.e. via any standard map update operation.
563 + */
564 + private class NonTransactionalCommit implements MapEntryValue {
565 + private final long version;
566 + private final Commit<? extends AtomixConsistentMapCommands.UpdateAndGet> commit;
567 +
568 + public NonTransactionalCommit(
569 + long version,
570 + Commit<? extends AtomixConsistentMapCommands.UpdateAndGet> commit) {
571 + this.version = version;
572 + this.commit = commit;
573 + }
574 +
575 + @Override
576 + public byte[] value() {
577 + return commit.operation().value();
578 + }
579 +
580 + @Override
581 + public long version() {
582 + return version;
583 + }
584 +
585 + @Override
586 + public void discard() {
587 + commit.close();
588 + }
589 + }
590 +
591 + /**
592 + * A {@code MapEntryValue} that is derived from updates submitted via a
593 + * transaction.
594 + */
595 + private class TransactionalCommit implements MapEntryValue {
596 + private final String key;
597 + private final long version;
598 + private final CountDownCompleter<Commit<? extends AtomixConsistentMapCommands.TransactionPrepare>> completer;
599 +
600 + public TransactionalCommit(
601 + String key,
602 + long version,
603 + CountDownCompleter<Commit<? extends AtomixConsistentMapCommands.TransactionPrepare>> commit) {
604 + this.key = key;
605 + this.version = version;
606 + this.completer = commit;
607 + }
608 +
609 + @Override
610 + public byte[] value() {
611 + TransactionalMapUpdate<String, byte[]> update = completer.object()
612 + .operation().transactionUpdate();
613 + return update.valueForKey(key);
614 + }
615 +
616 + @Override
617 + public long version() {
618 + return version;
619 + }
620 +
621 + @Override
622 + public void discard() {
623 + completer.countDown();
624 + }
625 + }
626 +}
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 static org.slf4j.LoggerFactory.getLogger;
19 +import io.atomix.copycat.client.session.Session;
20 +import io.atomix.copycat.server.Commit;
21 +import io.atomix.copycat.server.Snapshottable;
22 +import io.atomix.copycat.server.StateMachineExecutor;
23 +import io.atomix.copycat.server.session.SessionListener;
24 +import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
25 +import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
26 +import io.atomix.resource.ResourceStateMachine;
27 +
28 +import java.util.Arrays;
29 +import java.util.HashMap;
30 +import java.util.LinkedHashMap;
31 +import java.util.LinkedList;
32 +import java.util.List;
33 +import java.util.Map;
34 +import java.util.Optional;
35 +import java.util.Set;
36 +import java.util.concurrent.atomic.AtomicLong;
37 +import java.util.function.Supplier;
38 +import java.util.stream.Collectors;
39 +
40 +import org.onosproject.cluster.Leader;
41 +import org.onosproject.cluster.Leadership;
42 +import org.onosproject.cluster.NodeId;
43 +import org.onosproject.event.Change;
44 +import org.onosproject.store.serializers.KryoNamespaces;
45 +import org.onosproject.store.service.Serializer;
46 +import org.slf4j.Logger;
47 +
48 +import com.google.common.base.MoreObjects;
49 +import com.google.common.base.Objects;
50 +import com.google.common.collect.Lists;
51 +import com.google.common.collect.Maps;
52 +
53 +/**
54 + * State machine for {@link AtomixLeaderElector} resource.
55 + */
56 +public class AtomixLeaderElectorState extends ResourceStateMachine
57 + implements SessionListener, Snapshottable {
58 +
59 + private final Logger log = getLogger(getClass());
60 + private Map<String, AtomicLong> termCounters = new HashMap<>();
61 + private Map<String, ElectionState> elections = new HashMap<>();
62 + private final Map<Long, Commit<? extends AtomixLeaderElectorCommands.Listen>> listeners = new LinkedHashMap<>();
63 + private final Serializer serializer = Serializer.using(Arrays.asList(KryoNamespaces.API),
64 + ElectionState.class,
65 + Registration.class);
66 +
67 + @Override
68 + protected void configure(StateMachineExecutor executor) {
69 + // Notification
70 + executor.register(AtomixLeaderElectorCommands.Listen.class, this::listen);
71 + executor.register(AtomixLeaderElectorCommands.Unlisten.class, this::unlisten);
72 + // Commands
73 + executor.register(AtomixLeaderElectorCommands.Run.class, this::run);
74 + executor.register(AtomixLeaderElectorCommands.Withdraw.class, this::withdraw);
75 + executor.register(AtomixLeaderElectorCommands.Anoint.class, this::anoint);
76 + // Queries
77 + executor.register(AtomixLeaderElectorCommands.GetLeadership.class, this::leadership);
78 + executor.register(AtomixLeaderElectorCommands.GetAllLeaderships.class, this::allLeaderships);
79 + executor.register(AtomixLeaderElectorCommands.GetElectedTopics.class, this::electedTopics);
80 + }
81 +
82 + private void notifyLeadershipChange(Leadership previousLeadership, Leadership newLeadership) {
83 + Change<Leadership> change = new Change<>(previousLeadership, newLeadership);
84 + listeners.values().forEach(listener -> listener.session().publish("change", change));
85 + }
86 +
87 + @Override
88 + public void delete() {
89 + // Close and clear Listeners
90 + listeners.values().forEach(Commit::close);
91 + listeners.clear();
92 + }
93 +
94 + /**
95 + * Applies listen commits.
96 + *
97 + * @param commit listen commit
98 + */
99 + public void listen(Commit<? extends AtomixLeaderElectorCommands.Listen> commit) {
100 + if (listeners.putIfAbsent(commit.session().id(), commit) != null) {
101 + commit.close();
102 + }
103 + }
104 +
105 + /**
106 + * Applies unlisten commits.
107 + *
108 + * @param commit unlisten commit
109 + */
110 + public void unlisten(Commit<? extends AtomixLeaderElectorCommands.Unlisten> commit) {
111 + try {
112 + Commit<? extends AtomixLeaderElectorCommands.Listen> listener = listeners.remove(commit.session().id());
113 + if (listener != null) {
114 + listener.close();
115 + }
116 + } finally {
117 + commit.close();
118 + }
119 + }
120 +
121 + /**
122 + * Applies an {@link AtomixLeaderElectorCommands.Run} commit.
123 + * @param commit commit entry
124 + * @return topic leader. If no previous leader existed this is the node that just entered the race.
125 + */
126 + public Leadership run(Commit<? extends AtomixLeaderElectorCommands.Run> commit) {
127 + try {
128 + String topic = commit.operation().topic();
129 + Leadership oldLeadership = leadership(topic);
130 + Registration registration = new Registration(commit.operation().nodeId(), commit.session().id());
131 + elections.compute(topic, (k, v) -> {
132 + if (v == null) {
133 + return new ElectionState(registration, termCounter(topic)::incrementAndGet);
134 + } else {
135 + if (!v.isDuplicate(registration)) {
136 + return new ElectionState(v).addRegistration(registration, termCounter(topic)::incrementAndGet);
137 + } else {
138 + return v;
139 + }
140 + }
141 + });
142 + Leadership newLeadership = leadership(topic);
143 +
144 + if (!Objects.equal(oldLeadership, newLeadership)) {
145 + notifyLeadershipChange(oldLeadership, newLeadership);
146 + }
147 + return newLeadership;
148 + } finally {
149 + commit.close();
150 + }
151 + }
152 +
153 + /**
154 + * Applies an {@link AtomixLeaderElectorCommands.Withdraw} commit.
155 + * @param commit withdraw commit
156 + */
157 + public void withdraw(Commit<? extends AtomixLeaderElectorCommands.Withdraw> commit) {
158 + try {
159 + String topic = commit.operation().topic();
160 + Leadership oldLeadership = leadership(topic);
161 + elections.computeIfPresent(topic, (k, v) -> v.cleanup(commit.session(),
162 + termCounter(topic)::incrementAndGet));
163 + Leadership newLeadership = leadership(topic);
164 + if (!Objects.equal(oldLeadership, newLeadership)) {
165 + notifyLeadershipChange(oldLeadership, newLeadership);
166 + }
167 + } finally {
168 + commit.close();
169 + }
170 + }
171 +
172 + /**
173 + * Applies an {@link AtomixLeaderElectorCommands.Anoint} commit.
174 + * @param commit anoint commit
175 + * @return {@code true} if changes were made and the transfer occurred; {@code false} if it did not.
176 + */
177 + public boolean anoint(Commit<? extends AtomixLeaderElectorCommands.Anoint> commit) {
178 + try {
179 + String topic = commit.operation().topic();
180 + Leadership oldLeadership = leadership(topic);
181 + ElectionState electionState = elections.computeIfPresent(topic,
182 + (k, v) -> new ElectionState(v).transferLeadership(commit.operation().nodeId(), termCounter(topic)));
183 + Leadership newLeadership = leadership(topic);
184 + if (!Objects.equal(oldLeadership, newLeadership)) {
185 + notifyLeadershipChange(oldLeadership, newLeadership);
186 + }
187 + return (electionState != null &&
188 + electionState.leader() != null &&
189 + commit.operation().nodeId().equals(electionState.leader().nodeId()));
190 + } finally {
191 + commit.close();
192 + }
193 + }
194 +
195 + /**
196 + * Applies an {@link AtomixLeaderElectorCommands.GetLeadership} commit.
197 + * @param commit GetLeadership commit
198 + * @return leader
199 + */
200 + public Leadership leadership(Commit<? extends AtomixLeaderElectorCommands.GetLeadership> commit) {
201 + String topic = commit.operation().topic();
202 + try {
203 + return leadership(topic);
204 + } finally {
205 + commit.close();
206 + }
207 + }
208 +
209 + /**
210 + * Applies an {@link AtomixLeaderElectorCommands.GetElectedTopics} commit.
211 + * @param commit commit entry
212 + * @return set of topics for which the node is the leader
213 + */
214 + public Set<String> electedTopics(Commit<? extends AtomixLeaderElectorCommands.GetElectedTopics> commit) {
215 + try {
216 + NodeId nodeId = commit.operation().nodeId();
217 + return Maps.filterEntries(elections, e -> {
218 + Leader leader = leadership(e.getKey()).leader();
219 + return leader != null && leader.nodeId().equals(nodeId);
220 + }).keySet();
221 + } finally {
222 + commit.close();
223 + }
224 + }
225 +
226 + /**
227 + * Applies an {@link AtomixLeaderElectorCommands.GetAllLeaderships} commit.
228 + * @param commit GetAllLeaderships commit
229 + * @return topic to leader mapping
230 + */
231 + public Map<String, Leadership> allLeaderships(
232 + Commit<? extends AtomixLeaderElectorCommands.GetAllLeaderships> commit) {
233 + try {
234 + return Maps.transformEntries(elections, (k, v) -> leadership(k));
235 + } finally {
236 + commit.close();
237 + }
238 + }
239 +
240 + private Leadership leadership(String topic) {
241 + return new Leadership(topic,
242 + leader(topic),
243 + candidates(topic));
244 + }
245 +
246 + private Leader leader(String topic) {
247 + ElectionState electionState = elections.get(topic);
248 + return electionState == null ? null : electionState.leader();
249 + }
250 +
251 + private List<NodeId> candidates(String topic) {
252 + ElectionState electionState = elections.get(topic);
253 + return electionState == null ? new LinkedList<>() : electionState.candidates();
254 + }
255 +
256 + private void onSessionEnd(Session session) {
257 + Commit<? extends AtomixLeaderElectorCommands.Listen> listener = listeners.remove(session);
258 + if (listener != null) {
259 + listener.close();
260 + }
261 + Set<String> topics = elections.keySet();
262 + topics.forEach(topic -> {
263 + Leadership oldLeadership = leadership(topic);
264 + elections.compute(topic, (k, v) -> v.cleanup(session, termCounter(topic)::incrementAndGet));
265 + Leadership newLeadership = leadership(topic);
266 + if (!Objects.equal(oldLeadership, newLeadership)) {
267 + notifyLeadershipChange(oldLeadership, newLeadership);
268 + }
269 + });
270 + }
271 +
272 + private static class Registration {
273 + private final NodeId nodeId;
274 + private final long sessionId;
275 +
276 + public Registration(NodeId nodeId, long sessionId) {
277 + this.nodeId = nodeId;
278 + this.sessionId = sessionId;
279 + }
280 +
281 + public NodeId nodeId() {
282 + return nodeId;
283 + }
284 +
285 + public long sessionId() {
286 + return sessionId;
287 + }
288 +
289 + @Override
290 + public String toString() {
291 + return MoreObjects.toStringHelper(getClass())
292 + .add("nodeId", nodeId)
293 + .add("sessionId", sessionId)
294 + .toString();
295 + }
296 + }
297 +
298 + private static class ElectionState {
299 + final Registration leader;
300 + final long term;
301 + final long termStartTime;
302 + final List<Registration> registrations;
303 +
304 + public ElectionState(Registration registration, Supplier<Long> termCounter) {
305 + registrations = Arrays.asList(registration);
306 + term = termCounter.get();
307 + termStartTime = System.currentTimeMillis();
308 + leader = registration;
309 + }
310 +
311 + public ElectionState(ElectionState other) {
312 + registrations = Lists.newArrayList(other.registrations);
313 + leader = other.leader;
314 + term = other.term;
315 + termStartTime = other.termStartTime;
316 + }
317 +
318 + public ElectionState(List<Registration> registrations,
319 + Registration leader,
320 + long term,
321 + long termStartTime) {
322 + this.registrations = Lists.newArrayList(registrations);
323 + this.leader = leader;
324 + this.term = term;
325 + this.termStartTime = termStartTime;
326 + }
327 +
328 + public ElectionState cleanup(Session session, Supplier<Long> termCounter) {
329 + Optional<Registration> registration =
330 + registrations.stream().filter(r -> r.sessionId() == session.id()).findFirst();
331 + if (registration.isPresent()) {
332 + List<Registration> updatedRegistrations =
333 + registrations.stream()
334 + .filter(r -> r.sessionId() != session.id())
335 + .collect(Collectors.toList());
336 + if (leader.sessionId() == session.id()) {
337 + if (updatedRegistrations.size() > 0) {
338 + return new ElectionState(updatedRegistrations,
339 + updatedRegistrations.get(0),
340 + termCounter.get(),
341 + System.currentTimeMillis());
342 + } else {
343 + return new ElectionState(updatedRegistrations, null, term, termStartTime);
344 + }
345 + } else {
346 + return new ElectionState(updatedRegistrations, leader, term, termStartTime);
347 + }
348 + } else {
349 + return this;
350 + }
351 + }
352 +
353 + public boolean isDuplicate(Registration registration) {
354 + return registrations.stream().anyMatch(r -> r.sessionId() == registration.sessionId());
355 + }
356 +
357 + public Leader leader() {
358 + if (leader == null) {
359 + return null;
360 + } else {
361 + NodeId leaderNodeId = leader.nodeId();
362 + return new Leader(leaderNodeId, term, termStartTime);
363 + }
364 + }
365 +
366 + public List<NodeId> candidates() {
367 + return registrations.stream().map(registration -> registration.nodeId()).collect(Collectors.toList());
368 + }
369 +
370 + public ElectionState addRegistration(Registration registration, Supplier<Long> termCounter) {
371 + if (!registrations.stream().anyMatch(r -> r.sessionId() == registration.sessionId())) {
372 + List<Registration> updatedRegistrations = new LinkedList<>(registrations);
373 + updatedRegistrations.add(registration);
374 + boolean newLeader = leader == null;
375 + return new ElectionState(updatedRegistrations,
376 + newLeader ? registration : leader,
377 + newLeader ? termCounter.get() : term,
378 + newLeader ? System.currentTimeMillis() : termStartTime);
379 + }
380 + return this;
381 + }
382 +
383 + public ElectionState transferLeadership(NodeId nodeId, AtomicLong termCounter) {
384 + Registration newLeader = registrations.stream()
385 + .filter(r -> r.nodeId().equals(nodeId))
386 + .findFirst()
387 + .orElse(null);
388 + if (newLeader != null) {
389 + return new ElectionState(registrations,
390 + newLeader,
391 + termCounter.incrementAndGet(),
392 + System.currentTimeMillis());
393 + } else {
394 + return this;
395 + }
396 + }
397 + }
398 +
399 + @Override
400 + public void register(Session session) {
401 + }
402 +
403 + @Override
404 + public void unregister(Session session) {
405 + onSessionEnd(session);
406 + }
407 +
408 + @Override
409 + public void expire(Session session) {
410 + onSessionEnd(session);
411 + }
412 +
413 + @Override
414 + public void close(Session session) {
415 + onSessionEnd(session);
416 + }
417 +
418 + @Override
419 + public void snapshot(SnapshotWriter writer) {
420 + byte[] encodedTermCounters = serializer.encode(termCounters);
421 + writer.writeInt(encodedTermCounters.length);
422 + writer.write(encodedTermCounters);
423 + byte[] encodedElections = serializer.encode(elections);
424 + writer.writeInt(encodedElections.length);
425 + writer.write(encodedElections);
426 + log.info("Took state machine snapshot");
427 + }
428 +
429 + @Override
430 + public void install(SnapshotReader reader) {
431 + int encodedTermCountersSize = reader.readInt();
432 + byte[] encodedTermCounters = new byte[encodedTermCountersSize];
433 + reader.read(encodedTermCounters);
434 + termCounters = serializer.decode(encodedTermCounters);
435 + int encodedElectionsSize = reader.readInt();
436 + byte[] encodedElections = new byte[encodedElectionsSize];
437 + reader.read(encodedElections);
438 + elections = serializer.decode(encodedElections);
439 + log.info("Reinstated state machine from snapshot");
440 + }
441 +
442 + private AtomicLong termCounter(String topic) {
443 + return termCounters.computeIfAbsent(topic, k -> new AtomicLong(0));
444 + }
445 +}
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 io.atomix.resource.ResourceType;
19 +import static org.junit.Assert.*;
20 +
21 +import java.util.Arrays;
22 +import java.util.ConcurrentModificationException;
23 +import java.util.List;
24 +import java.util.concurrent.CompletionException;
25 +import java.util.stream.Collectors;
26 +
27 +import org.junit.Test;
28 +import org.onlab.util.Tools;
29 +import org.onosproject.store.service.MapEvent;
30 +import org.onosproject.store.service.MapEventListener;
31 +import org.onosproject.store.service.Versioned;
32 +
33 +import com.google.common.collect.Sets;
34 +
35 +/**
36 + * Unit tests for {@link AtomixConsistentMap}.
37 + */
38 +public class AtomixConsistentMapTest extends AtomixTestBase {
39 +
40 + @Override
41 + protected ResourceType resourceType() {
42 + return new ResourceType(AtomixConsistentMap.class);
43 + }
44 +
45 + /**
46 + * Tests various basic map operations.
47 + */
48 + @Test
49 + public void testBasicMapOperations() throws Throwable {
50 + basicMapOperationTests(1);
51 + clearTests();
52 + basicMapOperationTests(2);
53 + clearTests();
54 + basicMapOperationTests(3);
55 + }
56 +
57 + /**
58 + * Tests various map compute* operations on different cluster sizes.
59 + */
60 + @Test
61 + public void testMapComputeOperations() throws Throwable {
62 + mapComputeOperationTests(1);
63 + clearTests();
64 + mapComputeOperationTests(2);
65 + clearTests();
66 + mapComputeOperationTests(3);
67 + }
68 +
69 + /**
70 + * Tests map event notifications.
71 + */
72 + @Test
73 + public void testMapListeners() throws Throwable {
74 + mapListenerTests(1);
75 + clearTests();
76 + mapListenerTests(2);
77 + clearTests();
78 + mapListenerTests(3);
79 + }
80 +
81 + /**
82 + * Tests map transaction commit.
83 + */
84 + @Test
85 + public void testTransactionCommit() throws Throwable {
86 + transactionCommitTests(1);
87 + clearTests();
88 + transactionCommitTests(2);
89 + clearTests();
90 + transactionCommitTests(3);
91 + }
92 +
93 + /**
94 + * Tests map transaction rollback.
95 + */
96 + @Test
97 + public void testTransactionRollback() throws Throwable {
98 + transactionRollbackTests(1);
99 + clearTests();
100 + transactionRollbackTests(2);
101 + clearTests();
102 + transactionRollbackTests(3);
103 + }
104 +
105 + protected void basicMapOperationTests(int clusterSize) throws Throwable {
106 + createCopycatServers(clusterSize);
107 +
108 + final byte[] rawFooValue = Tools.getBytesUtf8("Hello foo!");
109 + final byte[] rawBarValue = Tools.getBytesUtf8("Hello bar!");
110 +
111 + AtomixConsistentMap map = createAtomixClient().get("test", AtomixConsistentMap.class).join();
112 +
113 + map.isEmpty().thenAccept(result -> {
114 + assertTrue(result);
115 + }).join();
116 +
117 + map.put("foo", rawFooValue).thenAccept(result -> {
118 + assertNull(result);
119 + }).join();
120 +
121 + map.size().thenAccept(result -> {
122 + assertTrue(result == 1);
123 + }).join();
124 +
125 + map.isEmpty().thenAccept(result -> {
126 + assertFalse(result);
127 + }).join();
128 +
129 + map.putIfAbsent("foo", "Hello foo again!".getBytes()).thenAccept(result -> {
130 + assertNotNull(result);
131 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawFooValue));
132 + }).join();
133 +
134 + map.putIfAbsent("bar", rawBarValue).thenAccept(result -> {
135 + assertNull(result);
136 + }).join();
137 +
138 + map.size().thenAccept(result -> {
139 + assertTrue(result == 2);
140 + }).join();
141 +
142 + map.keySet().thenAccept(result -> {
143 + assertTrue(result.size() == 2);
144 + assertTrue(result.containsAll(Sets.newHashSet("foo", "bar")));
145 + }).join();
146 +
147 + map.values().thenAccept(result -> {
148 + assertTrue(result.size() == 2);
149 + List<String> rawValues =
150 + result.stream().map(v -> Tools.toStringUtf8(v.value())).collect(Collectors.toList());
151 + assertTrue(rawValues.contains("Hello foo!"));
152 + assertTrue(rawValues.contains("Hello bar!"));
153 + }).join();
154 +
155 + map.entrySet().thenAccept(result -> {
156 + assertTrue(result.size() == 2);
157 + // TODO: check entries
158 + }).join();
159 +
160 + map.get("foo").thenAccept(result -> {
161 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawFooValue));
162 + }).join();
163 +
164 + map.remove("foo").thenAccept(result -> {
165 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawFooValue));
166 + }).join();
167 +
168 + map.containsKey("foo").thenAccept(result -> {
169 + assertFalse(result);
170 + }).join();
171 +
172 + map.get("foo").thenAccept(result -> {
173 + assertNull(result);
174 + }).join();
175 +
176 + map.get("bar").thenAccept(result -> {
177 + assertNotNull(result);
178 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawBarValue));
179 + }).join();
180 +
181 + map.containsKey("bar").thenAccept(result -> {
182 + assertTrue(result);
183 + }).join();
184 +
185 + map.size().thenAccept(result -> {
186 + assertTrue(result == 1);
187 + }).join();
188 +
189 + map.containsValue(rawBarValue).thenAccept(result -> {
190 + assertTrue(result);
191 + }).join();
192 +
193 + map.containsValue(rawFooValue).thenAccept(result -> {
194 + assertFalse(result);
195 + }).join();
196 +
197 + map.replace("bar", "Goodbye bar!".getBytes()).thenAccept(result -> {
198 + assertNotNull(result);
199 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawBarValue));
200 + }).join();
201 +
202 + map.replace("foo", "Goodbye foo!".getBytes()).thenAccept(result -> {
203 + assertNull(result);
204 + }).join();
205 +
206 + // try replace_if_value_match for a non-existent key
207 + map.replace("foo", "Goodbye foo!".getBytes(), rawFooValue).thenAccept(result -> {
208 + assertFalse(result);
209 + }).join();
210 +
211 + map.replace("bar", "Goodbye bar!".getBytes(), rawBarValue).thenAccept(result -> {
212 + assertTrue(result);
213 + }).join();
214 +
215 + map.replace("bar", "Goodbye bar!".getBytes(), rawBarValue).thenAccept(result -> {
216 + assertFalse(result);
217 + }).join();
218 +
219 + Versioned<byte[]> barValue = map.get("bar").join();
220 + map.replace("bar", barValue.version(), "Goodbye bar!".getBytes()).thenAccept(result -> {
221 + assertTrue(result);
222 + }).join();
223 +
224 + map.replace("bar", barValue.version(), rawBarValue).thenAccept(result -> {
225 + assertFalse(result);
226 + }).join();
227 +
228 + map.clear().join();
229 +
230 + map.size().thenAccept(result -> {
231 + assertTrue(result == 0);
232 + }).join();
233 + }
234 +
235 + public void mapComputeOperationTests(int clusterSize) throws Throwable {
236 + createCopycatServers(clusterSize);
237 + final byte[] value1 = Tools.getBytesUtf8("value1");
238 + final byte[] value2 = Tools.getBytesUtf8("value2");
239 + final byte[] value3 = Tools.getBytesUtf8("value3");
240 +
241 + AtomixConsistentMap map = createAtomixClient().get("test", AtomixConsistentMap.class).join();
242 +
243 + map.computeIfAbsent("foo", k -> value1).thenAccept(result -> {
244 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
245 + }).join();
246 +
247 + map.computeIfAbsent("foo", k -> value2).thenAccept(result -> {
248 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
249 + }).join();
250 +
251 + map.computeIfPresent("bar", (k, v) -> value2).thenAccept(result -> {
252 + assertNull(result);
253 + });
254 +
255 + map.computeIfPresent("foo", (k, v) -> value3).thenAccept(result -> {
256 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value3));
257 + }).join();
258 +
259 + map.computeIfPresent("foo", (k, v) -> null).thenAccept(result -> {
260 + assertNull(result);
261 + }).join();
262 +
263 + map.computeIf("foo", v -> v == null, (k, v) -> value1).thenAccept(result -> {
264 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
265 + }).join();
266 +
267 + map.compute("foo", (k, v) -> value2).thenAccept(result -> {
268 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value2));
269 + }).join();
270 + }
271 +
272 +
273 + protected void mapListenerTests(int clusterSize) throws Throwable {
274 + createCopycatServers(clusterSize);
275 + final byte[] value1 = Tools.getBytesUtf8("value1");
276 + final byte[] value2 = Tools.getBytesUtf8("value2");
277 + final byte[] value3 = Tools.getBytesUtf8("value3");
278 +
279 + AtomixConsistentMap map = createAtomixClient().get("test", AtomixConsistentMap.class).join();
280 + TestMapEventListener listener = new TestMapEventListener();
281 +
282 + // add listener; insert new value into map and verify an INSERT event is received.
283 + map.addListener(listener).join();
284 + map.put("foo", value1).join();
285 + assertNotNull(listener.event());
286 + assertEquals(MapEvent.Type.INSERT, listener.event().type());
287 + assertTrue(Arrays.equals(value1, listener.event().newValue().value()));
288 + listener.clearEvent();
289 +
290 + // remove listener and verify listener is not notified.
291 + map.removeListener(listener).join();
292 + map.put("foo", value2).join();
293 + assertNull(listener.event());
294 +
295 + // add the listener back and verify UPDATE events are received correctly
296 + map.addListener(listener).join();
297 + map.put("foo", value3).join();
298 + assertNotNull(listener.event());
299 + assertEquals(MapEvent.Type.UPDATE, listener.event().type());
300 + assertTrue(Arrays.equals(value3, listener.event().newValue().value()));
301 + listener.clearEvent();
302 +
303 + // perform a non-state changing operation and verify no events are received.
304 + map.putIfAbsent("foo", value1).join();
305 + assertNull(listener.event());
306 +
307 + // verify REMOVE events are received correctly.
308 + map.remove("foo").join();
309 + assertNotNull(listener.event());
310 + assertEquals(MapEvent.Type.REMOVE, listener.event().type());
311 + assertTrue(Arrays.equals(value3, listener.event().oldValue().value()));
312 + listener.clearEvent();
313 +
314 + // verify compute methods also generate events.
315 + map.computeIf("foo", v -> v == null, (k, v) -> value1).join();
316 + assertNotNull(listener.event());
317 + assertEquals(MapEvent.Type.INSERT, listener.event().type());
318 + assertTrue(Arrays.equals(value1, listener.event().newValue().value()));
319 + listener.clearEvent();
320 +
321 + map.compute("foo", (k, v) -> value2).join();
322 + assertNotNull(listener.event());
323 + assertEquals(MapEvent.Type.UPDATE, listener.event().type());
324 + assertTrue(Arrays.equals(value2, listener.event().newValue().value()));
325 + listener.clearEvent();
326 +
327 + map.computeIf("foo", v -> Arrays.equals(v, value2), (k, v) -> null).join();
328 + assertNotNull(listener.event());
329 + assertEquals(MapEvent.Type.REMOVE, listener.event().type());
330 + assertTrue(Arrays.equals(value2, listener.event().oldValue().value()));
331 + listener.clearEvent();
332 +
333 + map.removeListener(listener).join();
334 + }
335 +
336 + protected void transactionCommitTests(int clusterSize) throws Throwable {
337 + createCopycatServers(clusterSize);
338 + final byte[] value1 = Tools.getBytesUtf8("value1");
339 + final byte[] value2 = Tools.getBytesUtf8("value2");
340 +
341 + AtomixConsistentMap map = createAtomixClient().get("test", AtomixConsistentMap.class).join();
342 + TestMapEventListener listener = new TestMapEventListener();
343 +
344 + map.addListener(listener).join();
345 +
346 + MapUpdate<String, byte[]> update1 =
347 + MapUpdate.<String, byte[]>newBuilder().withType(MapUpdate.Type.PUT_IF_ABSENT)
348 + .withKey("foo")
349 + .withValue(value1)
350 + .build();
351 +
352 + TransactionalMapUpdate<String, byte[]> txMapUpdate =
353 + new TransactionalMapUpdate<>(TransactionId.from("tx1"), Arrays.asList(update1));
354 +
355 + map.prepare(txMapUpdate).thenAccept(result -> {
356 + assertEquals(PrepareResult.OK, result);
357 + }).join();
358 + assertNull(listener.event());
359 +
360 + map.size().thenAccept(result -> {
361 + assertTrue(result == 0);
362 + }).join();
363 +
364 + map.get("foo").thenAccept(result -> {
365 + assertNull(result);
366 + }).join();
367 +
368 + try {
369 + map.put("foo", value2).join();
370 + assertTrue(false);
371 + } catch (CompletionException e) {
372 + assertEquals(ConcurrentModificationException.class, e.getCause().getClass());
373 + }
374 +
375 + assertNull(listener.event());
376 +
377 + map.commit(txMapUpdate.transactionId()).join();
378 + assertNotNull(listener.event());
379 + assertEquals(MapEvent.Type.INSERT, listener.event().type());
380 + assertTrue(Arrays.equals(value1, listener.event().newValue().value()));
381 + listener.clearEvent();
382 +
383 + map.put("foo", value2).thenAccept(result -> {
384 + assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
385 + }).join();
386 + assertNotNull(listener.event());
387 + assertEquals(MapEvent.Type.UPDATE, listener.event().type());
388 + assertTrue(Arrays.equals(value2, listener.event().newValue().value()));
389 + listener.clearEvent();
390 + }
391 +
392 + protected void transactionRollbackTests(int clusterSize) throws Throwable {
393 + createCopycatServers(clusterSize);
394 + final byte[] value1 = Tools.getBytesUtf8("value1");
395 + final byte[] value2 = Tools.getBytesUtf8("value2");
396 +
397 + AtomixConsistentMap map = createAtomixClient().get("test", AtomixConsistentMap.class).join();
398 + TestMapEventListener listener = new TestMapEventListener();
399 +
400 + map.addListener(listener).join();
401 +
402 + MapUpdate<String, byte[]> update1 =
403 + MapUpdate.<String, byte[]>newBuilder().withType(MapUpdate.Type.PUT_IF_ABSENT)
404 + .withKey("foo")
405 + .withValue(value1)
406 + .build();
407 + TransactionalMapUpdate<String, byte[]> txMapUpdate =
408 + new TransactionalMapUpdate<>(TransactionId.from("tx1"), Arrays.asList(update1));
409 + map.prepare(txMapUpdate).thenAccept(result -> {
410 + assertEquals(PrepareResult.OK, result);
411 + }).join();
412 + assertNull(listener.event());
413 +
414 + map.rollback(txMapUpdate.transactionId()).join();
415 + assertNull(listener.event());
416 +
417 + map.get("foo").thenAccept(result -> {
418 + assertNull(result);
419 + }).join();
420 +
421 + map.put("foo", value2).thenAccept(result -> {
422 + assertNull(result);
423 + }).join();
424 + assertNotNull(listener.event());
425 + assertEquals(MapEvent.Type.INSERT, listener.event().type());
426 + assertTrue(Arrays.equals(value2, listener.event().newValue().value()));
427 + listener.clearEvent();
428 + }
429 +
430 + private static class TestMapEventListener implements MapEventListener<String, byte[]> {
431 +
432 + MapEvent<String, byte[]> event;
433 +
434 + @Override
435 + public void event(MapEvent<String, byte[]> event) {
436 + this.event = event;
437 + }
438 +
439 + public MapEvent<String, byte[]> event() {
440 + return event;
441 + }
442 +
443 + public void clearEvent() {
444 + event = null;
445 + }
446 + }
447 +}
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.LinkedList;
19 +import java.util.Queue;
20 +import java.util.concurrent.CompletableFuture;
21 +import java.util.function.Consumer;
22 +
23 +import org.junit.Test;
24 +
25 +import static org.junit.Assert.*;
26 +
27 +import org.onosproject.cluster.Leadership;
28 +import org.onosproject.cluster.NodeId;
29 +import org.onosproject.event.Change;
30 +
31 +import io.atomix.Atomix;
32 +import io.atomix.resource.ResourceType;
33 +
34 +/**
35 + * Unit tests for {@link AtomixLeaderElector}.
36 + */
37 +public class AtomixLeaderElectorTest extends AtomixTestBase {
38 +
39 + NodeId node1 = new NodeId("node1");
40 + NodeId node2 = new NodeId("node2");
41 + NodeId node3 = new NodeId("node3");
42 +
43 + @Override
44 + protected ResourceType resourceType() {
45 + return new ResourceType(AtomixLeaderElector.class);
46 + }
47 +
48 + @Test
49 + public void testRun() throws Throwable {
50 + leaderElectorRunTests(1);
51 + clearTests();
52 +// leaderElectorRunTests(2);
53 +// clearTests();
54 +// leaderElectorRunTests(3);
55 +// clearTests();
56 + }
57 +
58 + private void leaderElectorRunTests(int numServers) throws Throwable {
59 + createCopycatServers(numServers);
60 + Atomix client1 = createAtomixClient();
61 + AtomixLeaderElector elector1 = client1.get("test-elector", AtomixLeaderElector.class).join();
62 + elector1.run("foo", node1).thenAccept(result -> {
63 + assertEquals(node1, result.leaderNodeId());
64 + assertEquals(1, result.leader().term());
65 + assertEquals(1, result.candidates().size());
66 + assertEquals(node1, result.candidates().get(0));
67 + }).join();
68 + Atomix client2 = createAtomixClient();
69 + AtomixLeaderElector elector2 = client2.get("test-elector", AtomixLeaderElector.class).join();
70 + elector2.run("foo", node2).thenAccept(result -> {
71 + assertEquals(node1, result.leaderNodeId());
72 + assertEquals(1, result.leader().term());
73 + assertEquals(2, result.candidates().size());
74 + assertEquals(node1, result.candidates().get(0));
75 + assertEquals(node2, result.candidates().get(1));
76 + }).join();
77 + }
78 +
79 + @Test
80 + public void testWithdraw() throws Throwable {
81 + leaderElectorWithdrawTests(1);
82 + clearTests();
83 + leaderElectorWithdrawTests(2);
84 + clearTests();
85 + leaderElectorWithdrawTests(3);
86 + clearTests();
87 + }
88 +
89 + private void leaderElectorWithdrawTests(int numServers) throws Throwable {
90 + createCopycatServers(numServers);
91 + Atomix client1 = createAtomixClient();
92 + AtomixLeaderElector elector1 = client1.get("test-elector", AtomixLeaderElector.class).join();
93 + elector1.run("foo", node1).join();
94 + Atomix client2 = createAtomixClient();
95 + AtomixLeaderElector elector2 = client2.get("test-elector", AtomixLeaderElector.class).join();
96 + elector2.run("foo", node2).join();
97 +
98 + LeaderEventListener listener1 = new LeaderEventListener();
99 + elector1.addChangeListener(listener1).join();
100 +
101 + LeaderEventListener listener2 = new LeaderEventListener();
102 + elector2.addChangeListener(listener2).join();
103 +
104 + elector1.withdraw("foo").join();
105 +
106 + listener1.nextEvent().thenAccept(result -> {
107 + assertEquals(node2, result.newValue().leaderNodeId());
108 + assertEquals(2, result.newValue().leader().term());
109 + assertEquals(1, result.newValue().candidates().size());
110 + assertEquals(node2, result.newValue().candidates().get(0));
111 + }).join();
112 +
113 + listener2.nextEvent().thenAccept(result -> {
114 + assertEquals(node2, result.newValue().leaderNodeId());
115 + assertEquals(2, result.newValue().leader().term());
116 + assertEquals(1, result.newValue().candidates().size());
117 + assertEquals(node2, result.newValue().candidates().get(0));
118 + }).join();
119 + }
120 +
121 + @Test
122 + public void testAnoint() throws Throwable {
123 + leaderElectorAnointTests(1);
124 + clearTests();
125 + leaderElectorAnointTests(2);
126 + clearTests();
127 + leaderElectorAnointTests(3);
128 + clearTests();
129 + }
130 +
131 + private void leaderElectorAnointTests(int numServers) throws Throwable {
132 + createCopycatServers(numServers);
133 + Atomix client1 = createAtomixClient();
134 + AtomixLeaderElector elector1 = client1.get("test-elector", AtomixLeaderElector.class).join();
135 + Atomix client2 = createAtomixClient();
136 + AtomixLeaderElector elector2 = client2.get("test-elector", AtomixLeaderElector.class).join();
137 + Atomix client3 = createAtomixClient();
138 + AtomixLeaderElector elector3 = client3.get("test-elector", AtomixLeaderElector.class).join();
139 + elector1.run("foo", node1).join();
140 + elector2.run("foo", node2).join();
141 +
142 + LeaderEventListener listener1 = new LeaderEventListener();
143 + elector1.addChangeListener(listener1).join();
144 + LeaderEventListener listener2 = new LeaderEventListener();
145 + elector2.addChangeListener(listener2);
146 + LeaderEventListener listener3 = new LeaderEventListener();
147 + elector3.addChangeListener(listener3).join();
148 +
149 + elector3.anoint("foo", node3).thenAccept(result -> {
150 + assertFalse(result);
151 + }).join();
152 + assertFalse(listener1.hasEvent());
153 + assertFalse(listener2.hasEvent());
154 + assertFalse(listener3.hasEvent());
155 +
156 + elector3.anoint("foo", node2).thenAccept(result -> {
157 + assertTrue(result);
158 + }).join();
159 + assertTrue(listener1.hasEvent());
160 + assertTrue(listener2.hasEvent());
161 + assertTrue(listener3.hasEvent());
162 +
163 + listener1.nextEvent().thenAccept(result -> {
164 + assertEquals(node2, result.newValue().leaderNodeId());
165 + assertEquals(2, result.newValue().candidates().size());
166 + assertEquals(node1, result.newValue().candidates().get(0));
167 + assertEquals(node2, result.newValue().candidates().get(1));
168 + }).join();
169 + listener2.nextEvent().thenAccept(result -> {
170 + assertEquals(node2, result.newValue().leaderNodeId());
171 + assertEquals(2, result.newValue().candidates().size());
172 + assertEquals(node1, result.newValue().candidates().get(0));
173 + assertEquals(node2, result.newValue().candidates().get(1));
174 + }).join();
175 + listener3.nextEvent().thenAccept(result -> {
176 + assertEquals(node2, result.newValue().leaderNodeId());
177 + assertEquals(2, result.newValue().candidates().size());
178 + assertEquals(node1, result.newValue().candidates().get(0));
179 + assertEquals(node2, result.newValue().candidates().get(1));
180 + }).join();
181 + }
182 +
183 + @Test
184 + public void testLeaderSessionClose() throws Throwable {
185 + leaderElectorLeaderSessionCloseTests(1);
186 + clearTests();
187 + leaderElectorLeaderSessionCloseTests(2);
188 + clearTests();
189 + leaderElectorLeaderSessionCloseTests(3);
190 + clearTests();
191 + }
192 +
193 + private void leaderElectorLeaderSessionCloseTests(int numServers) throws Throwable {
194 + createCopycatServers(numServers);
195 + Atomix client1 = createAtomixClient();
196 + AtomixLeaderElector elector1 = client1.get("test-elector", AtomixLeaderElector.class).join();
197 + elector1.run("foo", node1).join();
198 + Atomix client2 = createAtomixClient();
199 + AtomixLeaderElector elector2 = client2.get("test-elector", AtomixLeaderElector.class).join();
200 + LeaderEventListener listener = new LeaderEventListener();
201 + elector2.run("foo", node2).join();
202 + elector2.addChangeListener(listener).join();
203 + client1.close();
204 + listener.nextEvent().thenAccept(result -> {
205 + assertEquals(node2, result.newValue().leaderNodeId());
206 + assertEquals(1, result.newValue().candidates().size());
207 + assertEquals(node2, result.newValue().candidates().get(0));
208 + }).join();
209 + }
210 +
211 + @Test
212 + public void testNonLeaderSessionClose() throws Throwable {
213 + leaderElectorNonLeaderSessionCloseTests(1);
214 + clearTests();
215 + leaderElectorNonLeaderSessionCloseTests(2);
216 + clearTests();
217 + leaderElectorNonLeaderSessionCloseTests(3);
218 + clearTests();
219 + }
220 +
221 + private void leaderElectorNonLeaderSessionCloseTests(int numServers) throws Throwable {
222 + createCopycatServers(numServers);
223 + Atomix client1 = createAtomixClient();
224 + AtomixLeaderElector elector1 = client1.get("test-elector", AtomixLeaderElector.class).join();
225 + elector1.run("foo", node1).join();
226 + Atomix client2 = createAtomixClient();
227 + AtomixLeaderElector elector2 = client2.get("test-elector", AtomixLeaderElector.class).join();
228 + LeaderEventListener listener = new LeaderEventListener();
229 + elector2.run("foo", node2).join();
230 + elector1.addChangeListener(listener).join();
231 + client2.close().join();
232 + listener.nextEvent().thenAccept(result -> {
233 + assertEquals(node1, result.newValue().leaderNodeId());
234 + assertEquals(1, result.newValue().candidates().size());
235 + assertEquals(node1, result.newValue().candidates().get(0));
236 + }).join();
237 + }
238 +
239 + @Test
240 + public void testQueries() throws Throwable {
241 + leaderElectorQueryTests(1);
242 + clearTests();
243 + leaderElectorQueryTests(2);
244 + clearTests();
245 + leaderElectorQueryTests(3);
246 + clearTests();
247 + }
248 +
249 + private void leaderElectorQueryTests(int numServers) throws Throwable {
250 + createCopycatServers(numServers);
251 + Atomix client1 = createAtomixClient();
252 + Atomix client2 = createAtomixClient();
253 + AtomixLeaderElector elector1 = client1.get("test-elector", AtomixLeaderElector.class).join();
254 + AtomixLeaderElector elector2 = client2.get("test-elector", AtomixLeaderElector.class).join();
255 + elector1.run("foo", node1).join();
256 + elector2.run("foo", node2).join();
257 + elector2.run("bar", node2).join();
258 + elector1.getElectedTopics(node1).thenAccept(result -> {
259 + assertEquals(1, result.size());
260 + assertTrue(result.contains("foo"));
261 + }).join();
262 + elector2.getElectedTopics(node1).thenAccept(result -> {
263 + assertEquals(1, result.size());
264 + assertTrue(result.contains("foo"));
265 + }).join();
266 + elector1.getLeadership("foo").thenAccept(result -> {
267 + assertEquals(node1, result.leaderNodeId());
268 + assertEquals(node1, result.candidates().get(0));
269 + assertEquals(node2, result.candidates().get(1));
270 + }).join();
271 + elector2.getLeadership("foo").thenAccept(result -> {
272 + assertEquals(node1, result.leaderNodeId());
273 + assertEquals(node1, result.candidates().get(0));
274 + assertEquals(node2, result.candidates().get(1));
275 + }).join();
276 + elector1.getLeadership("bar").thenAccept(result -> {
277 + assertEquals(node2, result.leaderNodeId());
278 + assertEquals(node2, result.candidates().get(0));
279 + }).join();
280 + elector2.getLeadership("bar").thenAccept(result -> {
281 + assertEquals(node2, result.leaderNodeId());
282 + assertEquals(node2, result.candidates().get(0));
283 + }).join();
284 + elector1.getLeaderships().thenAccept(result -> {
285 + assertEquals(2, result.size());
286 + Leadership fooLeadership = result.get("foo");
287 + assertEquals(node1, fooLeadership.leaderNodeId());
288 + assertEquals(node1, fooLeadership.candidates().get(0));
289 + assertEquals(node2, fooLeadership.candidates().get(1));
290 + Leadership barLeadership = result.get("bar");
291 + assertEquals(node2, barLeadership.leaderNodeId());
292 + assertEquals(node2, barLeadership.candidates().get(0));
293 + }).join();
294 + elector2.getLeaderships().thenAccept(result -> {
295 + assertEquals(2, result.size());
296 + Leadership fooLeadership = result.get("foo");
297 + assertEquals(node1, fooLeadership.leaderNodeId());
298 + assertEquals(node1, fooLeadership.candidates().get(0));
299 + assertEquals(node2, fooLeadership.candidates().get(1));
300 + Leadership barLeadership = result.get("bar");
301 + assertEquals(node2, barLeadership.leaderNodeId());
302 + assertEquals(node2, barLeadership.candidates().get(0));
303 + }).join();
304 + }
305 +
306 + private static class LeaderEventListener implements Consumer<Change<Leadership>> {
307 + Queue<Change<Leadership>> eventQueue = new LinkedList<>();
308 + CompletableFuture<Change<Leadership>> pendingFuture;
309 +
310 + @Override
311 + public void accept(Change<Leadership> change) {
312 + synchronized (this) {
313 + if (pendingFuture != null) {
314 + pendingFuture.complete(change);
315 + pendingFuture = null;
316 + } else {
317 + eventQueue.add(change);
318 + }
319 + }
320 + }
321 +
322 + public boolean hasEvent() {
323 + return !eventQueue.isEmpty();
324 + }
325 +
326 + public CompletableFuture<Change<Leadership>> nextEvent() {
327 + synchronized (this) {
328 + if (eventQueue.isEmpty()) {
329 + if (pendingFuture == null) {
330 + pendingFuture = new CompletableFuture<>();
331 + }
332 + return pendingFuture;
333 + } else {
334 + return CompletableFuture.completedFuture(eventQueue.poll());
335 + }
336 + }
337 + }
338 + }
339 +}
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 +}