Committed by
Gerrit Code Review
State machine implementations for various distributed primitives based on latest Copycat APIs
Change-Id: I622cc196aa1cdf072a5a0b100a5ffaaf71b07900
Showing
21 changed files
with
1692 additions
and
0 deletions
core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CatalystSerializers.java
0 → 100644
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 | +} |
This diff is collapsed. Click to expand it.
This diff is collapsed. Click to expand it.
This diff is collapsed. Click to expand it.
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 | +} |
This diff is collapsed. Click to expand it.
core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixValue.java
0 → 100644
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 | +} |
core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/MapUpdate.java
0 → 100644
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; |
This diff is collapsed. Click to expand it.
This diff is collapsed. Click to expand it.
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 | +} |
-
Please register or login to post a comment