Madan Jampani
Committed by Gerrit Code Review

ONOS-2440: Simplify DistributedQueue implementation by leveraging state change notification support

Change-Id: Id0a48f07535d8b7e1d0f964bd1c0623ca81d4605
...@@ -16,7 +16,6 @@ ...@@ -16,7 +16,6 @@
16 16
17 package org.onosproject.store.consistent.impl; 17 package org.onosproject.store.consistent.impl;
18 18
19 -import com.google.common.base.Charsets;
20 import com.google.common.collect.ArrayListMultimap; 19 import com.google.common.collect.ArrayListMultimap;
21 import com.google.common.collect.ImmutableList; 20 import com.google.common.collect.ImmutableList;
22 import com.google.common.collect.ImmutableSet; 21 import com.google.common.collect.ImmutableSet;
...@@ -49,8 +48,6 @@ import org.apache.felix.scr.annotations.ReferenceCardinality; ...@@ -49,8 +48,6 @@ import org.apache.felix.scr.annotations.ReferenceCardinality;
49 import org.apache.felix.scr.annotations.ReferencePolicy; 48 import org.apache.felix.scr.annotations.ReferencePolicy;
50 import org.apache.felix.scr.annotations.Service; 49 import org.apache.felix.scr.annotations.Service;
51 50
52 -import static org.onlab.util.Tools.groupedThreads;
53 -
54 import org.onosproject.app.ApplicationEvent; 51 import org.onosproject.app.ApplicationEvent;
55 import org.onosproject.app.ApplicationListener; 52 import org.onosproject.app.ApplicationListener;
56 import org.onosproject.app.ApplicationService; 53 import org.onosproject.app.ApplicationService;
...@@ -61,7 +58,6 @@ import org.onosproject.core.IdGenerator; ...@@ -61,7 +58,6 @@ import org.onosproject.core.IdGenerator;
61 import org.onosproject.store.cluster.impl.ClusterDefinitionManager; 58 import org.onosproject.store.cluster.impl.ClusterDefinitionManager;
62 import org.onosproject.store.cluster.impl.NodeInfo; 59 import org.onosproject.store.cluster.impl.NodeInfo;
63 import org.onosproject.store.cluster.messaging.ClusterCommunicationService; 60 import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
64 -import org.onosproject.store.cluster.messaging.MessageSubject;
65 import org.onosproject.store.ecmap.EventuallyConsistentMapBuilderImpl; 61 import org.onosproject.store.ecmap.EventuallyConsistentMapBuilderImpl;
66 import org.onosproject.store.service.AtomicCounterBuilder; 62 import org.onosproject.store.service.AtomicCounterBuilder;
67 import org.onosproject.store.service.AtomicValueBuilder; 63 import org.onosproject.store.service.AtomicValueBuilder;
...@@ -86,7 +82,6 @@ import java.util.Map; ...@@ -86,7 +82,6 @@ import java.util.Map;
86 import java.util.Set; 82 import java.util.Set;
87 import java.util.concurrent.CompletableFuture; 83 import java.util.concurrent.CompletableFuture;
88 import java.util.concurrent.ExecutionException; 84 import java.util.concurrent.ExecutionException;
89 -import java.util.concurrent.ExecutorService;
90 import java.util.concurrent.Executors; 85 import java.util.concurrent.Executors;
91 import java.util.concurrent.TimeUnit; 86 import java.util.concurrent.TimeUnit;
92 import java.util.concurrent.TimeoutException; 87 import java.util.concurrent.TimeoutException;
...@@ -112,8 +107,6 @@ public class DatabaseManager implements StorageService, StorageAdminService { ...@@ -112,8 +107,6 @@ public class DatabaseManager implements StorageService, StorageAdminService {
112 private static final int RAFT_ELECTION_TIMEOUT_MILLIS = 3000; 107 private static final int RAFT_ELECTION_TIMEOUT_MILLIS = 3000;
113 private static final int DATABASE_OPERATION_TIMEOUT_MILLIS = 5000; 108 private static final int DATABASE_OPERATION_TIMEOUT_MILLIS = 5000;
114 109
115 - protected static final MessageSubject QUEUE_UPDATED_TOPIC = new MessageSubject("distributed-queue-updated");
116 -
117 private ClusterCoordinator coordinator; 110 private ClusterCoordinator coordinator;
118 protected PartitionedDatabase partitionedDatabase; 111 protected PartitionedDatabase partitionedDatabase;
119 protected Database inMemoryDatabase; 112 protected Database inMemoryDatabase;
...@@ -122,15 +115,12 @@ public class DatabaseManager implements StorageService, StorageAdminService { ...@@ -122,15 +115,12 @@ public class DatabaseManager implements StorageService, StorageAdminService {
122 private TransactionManager transactionManager; 115 private TransactionManager transactionManager;
123 private final IdGenerator transactionIdGenerator = () -> RandomUtils.nextLong(); 116 private final IdGenerator transactionIdGenerator = () -> RandomUtils.nextLong();
124 117
125 - private ExecutorService eventDispatcher;
126 - private ExecutorService queuePollExecutor;
127 private ApplicationListener appListener = new InternalApplicationListener(); 118 private ApplicationListener appListener = new InternalApplicationListener();
128 119
129 private final Multimap<String, DefaultAsyncConsistentMap> maps = 120 private final Multimap<String, DefaultAsyncConsistentMap> maps =
130 Multimaps.synchronizedMultimap(ArrayListMultimap.create()); 121 Multimaps.synchronizedMultimap(ArrayListMultimap.create());
131 private final Multimap<ApplicationId, DefaultAsyncConsistentMap> mapsByApplication = 122 private final Multimap<ApplicationId, DefaultAsyncConsistentMap> mapsByApplication =
132 Multimaps.synchronizedMultimap(ArrayListMultimap.create()); 123 Multimaps.synchronizedMultimap(ArrayListMultimap.create());
133 - private final Map<String, DefaultDistributedQueue> queues = Maps.newConcurrentMap();
134 124
135 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY) 125 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
136 protected ClusterService clusterService; 126 protected ClusterService clusterService;
...@@ -237,21 +227,6 @@ public class DatabaseManager implements StorageService, StorageAdminService { ...@@ -237,21 +227,6 @@ public class DatabaseManager implements StorageService, StorageAdminService {
237 transactionManager = new TransactionManager(partitionedDatabase, consistentMapBuilder()); 227 transactionManager = new TransactionManager(partitionedDatabase, consistentMapBuilder());
238 partitionedDatabase.setTransactionManager(transactionManager); 228 partitionedDatabase.setTransactionManager(transactionManager);
239 229
240 - eventDispatcher = Executors.newSingleThreadExecutor(
241 - groupedThreads("onos/store/manager", "map-event-dispatcher"));
242 -
243 - queuePollExecutor = Executors.newFixedThreadPool(4,
244 - groupedThreads("onos/store/manager", "queue-poll-handler"));
245 -
246 - clusterCommunicator.<String>addSubscriber(QUEUE_UPDATED_TOPIC,
247 - data -> new String(data, Charsets.UTF_8),
248 - name -> {
249 - DefaultDistributedQueue q = queues.get(name);
250 - if (q != null) {
251 - q.tryPoll();
252 - }
253 - },
254 - queuePollExecutor);
255 log.info("Started"); 230 log.info("Started");
256 } 231 }
257 232
...@@ -277,13 +252,10 @@ public class DatabaseManager implements StorageService, StorageAdminService { ...@@ -277,13 +252,10 @@ public class DatabaseManager implements StorageService, StorageAdminService {
277 log.info("Successfully closed databases."); 252 log.info("Successfully closed databases.");
278 } 253 }
279 }); 254 });
280 - clusterCommunicator.removeSubscriber(QUEUE_UPDATED_TOPIC);
281 maps.values().forEach(this::unregisterMap); 255 maps.values().forEach(this::unregisterMap);
282 if (applicationService != null) { 256 if (applicationService != null) {
283 applicationService.removeListener(appListener); 257 applicationService.removeListener(appListener);
284 } 258 }
285 - eventDispatcher.shutdown();
286 - queuePollExecutor.shutdown();
287 log.info("Stopped"); 259 log.info("Stopped");
288 } 260 }
289 261
...@@ -467,13 +439,6 @@ public class DatabaseManager implements StorageService, StorageAdminService { ...@@ -467,13 +439,6 @@ public class DatabaseManager implements StorageService, StorageAdminService {
467 } 439 }
468 } 440 }
469 441
470 - protected <E> void registerQueue(DefaultDistributedQueue<E> queue) {
471 - // TODO: Support multiple local instances of the same queue.
472 - if (queues.putIfAbsent(queue.name(), queue) != null) {
473 - throw new IllegalStateException("Queue by name " + queue.name() + " already exists");
474 - }
475 - }
476 -
477 private class InternalApplicationListener implements ApplicationListener { 442 private class InternalApplicationListener implements ApplicationListener {
478 @Override 443 @Override
479 public void event(ApplicationEvent event) { 444 public void event(ApplicationEvent event) {
......
...@@ -21,7 +21,6 @@ import java.util.Map; ...@@ -21,7 +21,6 @@ import java.util.Map;
21 import java.util.Set; 21 import java.util.Set;
22 import java.util.concurrent.CompletableFuture; 22 import java.util.concurrent.CompletableFuture;
23 23
24 -import org.onosproject.cluster.NodeId;
25 import org.onosproject.store.service.Transaction; 24 import org.onosproject.store.service.Transaction;
26 import org.onosproject.store.service.Versioned; 25 import org.onosproject.store.service.Versioned;
27 26
...@@ -168,17 +167,16 @@ public interface DatabaseProxy<K, V> { ...@@ -168,17 +167,16 @@ public interface DatabaseProxy<K, V> {
168 * Inserts an entry into the queue. 167 * Inserts an entry into the queue.
169 * @param queueName queue name 168 * @param queueName queue name
170 * @param entry queue entry 169 * @param entry queue entry
171 - * @return set of nodes to notify about the queue update 170 + * @return void future
172 */ 171 */
173 - CompletableFuture<Set<NodeId>> queuePush(String queueName, byte[] entry); 172 + CompletableFuture<Void> queuePush(String queueName, byte[] entry);
174 173
175 /** 174 /**
176 * Removes an entry from the queue if the queue is non-empty. 175 * Removes an entry from the queue if the queue is non-empty.
177 * @param queueName queue name 176 * @param queueName queue name
178 - * @param nodeId If the queue is empty the identifier of node to notify when an entry becomes available 177 + * @return entry future. Can be completed with null if queue is empty
179 - * @return entry. Can be null if queue is empty
180 */ 178 */
181 - CompletableFuture<byte[]> queuePop(String queueName, NodeId nodeId); 179 + CompletableFuture<byte[]> queuePop(String queueName);
182 180
183 /** 181 /**
184 * Returns but does not remove an entry from the queue. 182 * Returns but does not remove an entry from the queue.
......
...@@ -21,7 +21,6 @@ import java.util.Map; ...@@ -21,7 +21,6 @@ import java.util.Map;
21 import java.util.Map.Entry; 21 import java.util.Map.Entry;
22 import java.util.Set; 22 import java.util.Set;
23 23
24 -import org.onosproject.cluster.NodeId;
25 import org.onosproject.store.service.Transaction; 24 import org.onosproject.store.service.Transaction;
26 import org.onosproject.store.service.Versioned; 25 import org.onosproject.store.service.Versioned;
27 26
...@@ -93,10 +92,10 @@ public interface DatabaseState<K, V> { ...@@ -93,10 +92,10 @@ public interface DatabaseState<K, V> {
93 byte[] queuePeek(String queueName); 92 byte[] queuePeek(String queueName);
94 93
95 @Command 94 @Command
96 - byte[] queuePop(String queueName, NodeId requestor); 95 + byte[] queuePop(String queueName);
97 96
98 @Command 97 @Command
99 - Set<NodeId> queuePush(String queueName, byte[] entry); 98 + void queuePush(String queueName, byte[] entry);
100 99
101 @Query 100 @Query
102 Long counterGet(String counterName); 101 Long counterGet(String counterName);
......
...@@ -47,7 +47,7 @@ import java.util.function.Predicate; ...@@ -47,7 +47,7 @@ import java.util.function.Predicate;
47 import java.util.stream.Collectors; 47 import java.util.stream.Collectors;
48 48
49 import static com.google.common.base.Preconditions.checkNotNull; 49 import static com.google.common.base.Preconditions.checkNotNull;
50 -import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.MAP; 50 +import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.MAP_UPDATE;
51 import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.TX_COMMIT; 51 import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.TX_COMMIT;
52 import static org.slf4j.LoggerFactory.getLogger; 52 import static org.slf4j.LoggerFactory.getLogger;
53 53
...@@ -122,7 +122,7 @@ public class DefaultAsyncConsistentMap<K, V> implements AsyncConsistentMap<K, V ...@@ -122,7 +122,7 @@ public class DefaultAsyncConsistentMap<K, V> implements AsyncConsistentMap<K, V
122 this.purgeOnUninstall = purgeOnUninstall; 122 this.purgeOnUninstall = purgeOnUninstall;
123 this.database.registerConsumer(update -> { 123 this.database.registerConsumer(update -> {
124 SharedExecutors.getSingleThreadExecutor().execute(() -> { 124 SharedExecutors.getSingleThreadExecutor().execute(() -> {
125 - if (update.target() == MAP) { 125 + if (update.target() == MAP_UPDATE) {
126 Result<UpdateResult<String, byte[]>> result = update.output(); 126 Result<UpdateResult<String, byte[]>> result = update.output();
127 if (result.success() && result.value().mapName().equals(name)) { 127 if (result.success() && result.value().mapName().equals(name)) {
128 MapEvent<K, V> mapEvent = result.value().<K, V>map(this::dK, serializer::decode).toMapEvent(); 128 MapEvent<K, V> mapEvent = result.value().<K, V>map(this::dK, serializer::decode).toMapEvent();
......
...@@ -30,7 +30,6 @@ import java.util.concurrent.CompletableFuture; ...@@ -30,7 +30,6 @@ import java.util.concurrent.CompletableFuture;
30 import java.util.function.Consumer; 30 import java.util.function.Consumer;
31 import java.util.function.Supplier; 31 import java.util.function.Supplier;
32 32
33 -import org.onosproject.cluster.NodeId;
34 import org.onosproject.store.service.Transaction; 33 import org.onosproject.store.service.Transaction;
35 import org.onosproject.store.service.Versioned; 34 import org.onosproject.store.service.Versioned;
36 35
...@@ -159,13 +158,13 @@ public class DefaultDatabase extends AbstractResource<Database> implements Datab ...@@ -159,13 +158,13 @@ public class DefaultDatabase extends AbstractResource<Database> implements Datab
159 } 158 }
160 159
161 @Override 160 @Override
162 - public CompletableFuture<Set<NodeId>> queuePush(String queueName, byte[] entry) { 161 + public CompletableFuture<Void> queuePush(String queueName, byte[] entry) {
163 return checkOpen(() -> proxy.queuePush(queueName, entry)); 162 return checkOpen(() -> proxy.queuePush(queueName, entry));
164 } 163 }
165 164
166 @Override 165 @Override
167 - public CompletableFuture<byte[]> queuePop(String queueName, NodeId nodeId) { 166 + public CompletableFuture<byte[]> queuePop(String queueName) {
168 - return checkOpen(() -> proxy.queuePop(queueName, nodeId)); 167 + return checkOpen(() -> proxy.queuePop(queueName));
169 } 168 }
170 169
171 @Override 170 @Override
......
...@@ -18,7 +18,6 @@ package org.onosproject.store.consistent.impl; ...@@ -18,7 +18,6 @@ package org.onosproject.store.consistent.impl;
18 18
19 import java.util.Arrays; 19 import java.util.Arrays;
20 import java.util.Collection; 20 import java.util.Collection;
21 -import java.util.HashSet;
22 import java.util.LinkedList; 21 import java.util.LinkedList;
23 import java.util.Map; 22 import java.util.Map;
24 import java.util.Map.Entry; 23 import java.util.Map.Entry;
...@@ -27,7 +26,6 @@ import java.util.concurrent.atomic.AtomicLong; ...@@ -27,7 +26,6 @@ import java.util.concurrent.atomic.AtomicLong;
27 import java.util.stream.Collectors; 26 import java.util.stream.Collectors;
28 import java.util.Set; 27 import java.util.Set;
29 28
30 -import org.onosproject.cluster.NodeId;
31 import org.onosproject.store.service.DatabaseUpdate; 29 import org.onosproject.store.service.DatabaseUpdate;
32 import org.onosproject.store.service.Transaction; 30 import org.onosproject.store.service.Transaction;
33 import org.onosproject.store.service.Versioned; 31 import org.onosproject.store.service.Versioned;
...@@ -48,7 +46,6 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> { ...@@ -48,7 +46,6 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> {
48 private Map<String, AtomicLong> counters; 46 private Map<String, AtomicLong> counters;
49 private Map<String, Map<String, Versioned<byte[]>>> maps; 47 private Map<String, Map<String, Versioned<byte[]>>> maps;
50 private Map<String, Queue<byte[]>> queues; 48 private Map<String, Queue<byte[]>> queues;
51 - private Map<String, Set<NodeId>> queueUpdateNotificationTargets;
52 49
53 /** 50 /**
54 * This locks map has a structure similar to the "tables" map above and 51 * This locks map has a structure similar to the "tables" map above and
...@@ -85,11 +82,6 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> { ...@@ -85,11 +82,6 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> {
85 queues = Maps.newConcurrentMap(); 82 queues = Maps.newConcurrentMap();
86 context.put("queues", queues); 83 context.put("queues", queues);
87 } 84 }
88 - queueUpdateNotificationTargets = context.get("queueUpdateNotificationTargets");
89 - if (queueUpdateNotificationTargets == null) {
90 - queueUpdateNotificationTargets = Maps.newConcurrentMap();
91 - context.put("queueUpdateNotificationTargets", queueUpdateNotificationTargets);
92 - }
93 nextVersion = context.get("nextVersion"); 85 nextVersion = context.get("nextVersion");
94 if (nextVersion == null) { 86 if (nextVersion == null) {
95 nextVersion = new Long(0); 87 nextVersion = new Long(0);
...@@ -214,27 +206,17 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> { ...@@ -214,27 +206,17 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> {
214 206
215 @Override 207 @Override
216 public byte[] queuePeek(String queueName) { 208 public byte[] queuePeek(String queueName) {
217 - Queue<byte[]> queue = getQueue(queueName); 209 + return getQueue(queueName).peek();
218 - return queue.peek();
219 } 210 }
220 211
221 @Override 212 @Override
222 - public byte[] queuePop(String queueName, NodeId requestor) { 213 + public byte[] queuePop(String queueName) {
223 - Queue<byte[]> queue = getQueue(queueName); 214 + return getQueue(queueName).poll();
224 - if (queue.size() == 0 && requestor != null) {
225 - getQueueUpdateNotificationTargets(queueName).add(requestor);
226 - return null;
227 - } else {
228 - return queue.remove();
229 - }
230 } 215 }
231 216
232 @Override 217 @Override
233 - public Set<NodeId> queuePush(String queueName, byte[] entry) { 218 + public void queuePush(String queueName, byte[] entry) {
234 - getQueue(queueName).add(entry); 219 + getQueue(queueName).offer(entry);
235 - Set<NodeId> notifyList = ImmutableSet.copyOf(getQueueUpdateNotificationTargets(queueName));
236 - getQueueUpdateNotificationTargets(queueName).clear();
237 - return notifyList;
238 } 220 }
239 221
240 @Override 222 @Override
...@@ -289,10 +271,6 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> { ...@@ -289,10 +271,6 @@ public class DefaultDatabaseState implements DatabaseState<String, byte[]> {
289 return queues.computeIfAbsent(queueName, name -> new LinkedList<>()); 271 return queues.computeIfAbsent(queueName, name -> new LinkedList<>());
290 } 272 }
291 273
292 - private Set<NodeId> getQueueUpdateNotificationTargets(String queueName) {
293 - return queueUpdateNotificationTargets.computeIfAbsent(queueName, name -> new HashSet<>());
294 - }
295 -
296 private boolean isUpdatePossible(DatabaseUpdate update) { 274 private boolean isUpdatePossible(DatabaseUpdate update) {
297 Versioned<byte[]> existingEntry = mapGet(update.mapName(), update.key()); 275 Versioned<byte[]> existingEntry = mapGet(update.mapName(), update.key());
298 switch (update.type()) { 276 switch (update.type()) {
......
...@@ -17,15 +17,16 @@ package org.onosproject.store.consistent.impl; ...@@ -17,15 +17,16 @@ package org.onosproject.store.consistent.impl;
17 17
18 import com.google.common.collect.Sets; 18 import com.google.common.collect.Sets;
19 import com.google.common.util.concurrent.Futures; 19 import com.google.common.util.concurrent.Futures;
20 -import org.onosproject.cluster.NodeId; 20 +
21 +import org.onlab.util.SharedExecutors;
21 import org.onosproject.store.service.DistributedQueue; 22 import org.onosproject.store.service.DistributedQueue;
22 import org.onosproject.store.service.Serializer; 23 import org.onosproject.store.service.Serializer;
23 24
25 +import java.util.List;
24 import java.util.Set; 26 import java.util.Set;
25 import java.util.concurrent.CompletableFuture; 27 import java.util.concurrent.CompletableFuture;
26 -import java.util.function.Consumer;
27 -
28 import static com.google.common.base.Preconditions.checkNotNull; 28 import static com.google.common.base.Preconditions.checkNotNull;
29 +import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.QUEUE_PUSH;
29 30
30 /** 31 /**
31 * DistributedQueue implementation that provides FIFO ordering semantics. 32 * DistributedQueue implementation that provides FIFO ordering semantics.
...@@ -37,9 +38,7 @@ public class DefaultDistributedQueue<E> implements DistributedQueue<E> { ...@@ -37,9 +38,7 @@ public class DefaultDistributedQueue<E> implements DistributedQueue<E> {
37 private final String name; 38 private final String name;
38 private final Database database; 39 private final Database database;
39 private final Serializer serializer; 40 private final Serializer serializer;
40 - private final NodeId localNodeId;
41 private final Set<CompletableFuture<E>> pendingFutures = Sets.newIdentityHashSet(); 41 private final Set<CompletableFuture<E>> pendingFutures = Sets.newIdentityHashSet();
42 - private final Consumer<Set<NodeId>> notifyConsumers;
43 42
44 private static final String PRIMITIVE_NAME = "distributedQueue"; 43 private static final String PRIMITIVE_NAME = "distributedQueue";
45 private static final String SIZE = "size"; 44 private static final String SIZE = "size";
...@@ -53,66 +52,59 @@ public class DefaultDistributedQueue<E> implements DistributedQueue<E> { ...@@ -53,66 +52,59 @@ public class DefaultDistributedQueue<E> implements DistributedQueue<E> {
53 public DefaultDistributedQueue(String name, 52 public DefaultDistributedQueue(String name,
54 Database database, 53 Database database,
55 Serializer serializer, 54 Serializer serializer,
56 - NodeId localNodeId, 55 + boolean meteringEnabled) {
57 - boolean meteringEnabled,
58 - Consumer<Set<NodeId>> notifyConsumers) {
59 this.name = checkNotNull(name, "queue name cannot be null"); 56 this.name = checkNotNull(name, "queue name cannot be null");
60 this.database = checkNotNull(database, "database cannot be null"); 57 this.database = checkNotNull(database, "database cannot be null");
61 this.serializer = checkNotNull(serializer, "serializer cannot be null"); 58 this.serializer = checkNotNull(serializer, "serializer cannot be null");
62 - this.localNodeId = localNodeId;
63 - this.notifyConsumers = notifyConsumers;
64 this.monitor = new MeteringAgent(PRIMITIVE_NAME, name, meteringEnabled); 59 this.monitor = new MeteringAgent(PRIMITIVE_NAME, name, meteringEnabled);
65 - 60 + this.database.registerConsumer(update -> {
61 + SharedExecutors.getSingleThreadExecutor().execute(() -> {
62 + if (update.target() == QUEUE_PUSH) {
63 + List<Object> input = update.input();
64 + String queueName = (String) input.get(0);
65 + if (queueName.equals(name)) {
66 + tryPoll();
67 + }
68 + }
69 + });
70 + });
66 } 71 }
67 72
68 @Override 73 @Override
69 public long size() { 74 public long size() {
70 final MeteringAgent.Context timer = monitor.startTimer(SIZE); 75 final MeteringAgent.Context timer = monitor.startTimer(SIZE);
71 - try { 76 + return Futures.getUnchecked(database.queueSize(name).whenComplete((r, e) -> timer.stop()));
72 - return Futures.getUnchecked(database.queueSize(name));
73 - } finally {
74 - timer.stop();
75 - }
76 } 77 }
77 78
78 @Override 79 @Override
79 public void push(E entry) { 80 public void push(E entry) {
81 + checkNotNull(entry, ERROR_NULL_ENTRY);
80 final MeteringAgent.Context timer = monitor.startTimer(PUSH); 82 final MeteringAgent.Context timer = monitor.startTimer(PUSH);
81 - try { 83 + Futures.getUnchecked(database.queuePush(name, serializer.encode(entry))
82 - checkNotNull(entry, ERROR_NULL_ENTRY); 84 + .whenComplete((r, e) -> timer.stop()));
83 - Futures.getUnchecked(database.queuePush(name, serializer.encode(entry))
84 - .thenAccept(notifyConsumers)
85 - .thenApply(v -> null));
86 - } finally {
87 - timer.stop();
88 - }
89 } 85 }
90 86
91 @Override 87 @Override
92 public CompletableFuture<E> pop() { 88 public CompletableFuture<E> pop() {
93 final MeteringAgent.Context timer = monitor.startTimer(POP); 89 final MeteringAgent.Context timer = monitor.startTimer(POP);
94 - return database.queuePop(name, localNodeId) 90 + return database.queuePop(name)
91 + .whenComplete((r, e) -> timer.stop())
95 .thenCompose(v -> { 92 .thenCompose(v -> {
96 if (v != null) { 93 if (v != null) {
97 return CompletableFuture.completedFuture(serializer.decode(v)); 94 return CompletableFuture.completedFuture(serializer.decode(v));
98 - } else {
99 - CompletableFuture<E> newPendingFuture = new CompletableFuture<>();
100 - pendingFutures.add(newPendingFuture);
101 - return newPendingFuture;
102 } 95 }
103 - }) 96 + CompletableFuture<E> newPendingFuture = new CompletableFuture<>();
104 - .whenComplete((r, e) -> timer.stop()); 97 + pendingFutures.add(newPendingFuture);
98 + return newPendingFuture;
99 + });
105 } 100 }
106 101
107 @Override 102 @Override
108 public E peek() { 103 public E peek() {
109 final MeteringAgent.Context timer = monitor.startTimer(PEEK); 104 final MeteringAgent.Context timer = monitor.startTimer(PEEK);
110 - try { 105 + return Futures.getUnchecked(database.queuePeek(name)
111 - return Futures.getUnchecked(database.queuePeek(name) 106 + .thenApply(v -> v != null ? serializer.<E>decode(v) : null)
112 - .thenApply(v -> v != null ? serializer.decode(v) : null)); 107 + .whenComplete((r, e) -> timer.stop()));
113 - } finally {
114 - timer.stop();
115 - }
116 } 108 }
117 109
118 public String name() { 110 public String name() {
...@@ -122,7 +114,7 @@ public class DefaultDistributedQueue<E> implements DistributedQueue<E> { ...@@ -122,7 +114,7 @@ public class DefaultDistributedQueue<E> implements DistributedQueue<E> {
122 protected void tryPoll() { 114 protected void tryPoll() {
123 Set<CompletableFuture<E>> completedFutures = Sets.newHashSet(); 115 Set<CompletableFuture<E>> completedFutures = Sets.newHashSet();
124 for (CompletableFuture<E> future : pendingFutures) { 116 for (CompletableFuture<E> future : pendingFutures) {
125 - E entry = Futures.getUnchecked(database.queuePop(name, localNodeId) 117 + E entry = Futures.getUnchecked(database.queuePop(name)
126 .thenApply(v -> v != null ? serializer.decode(v) : null)); 118 .thenApply(v -> v != null ? serializer.decode(v) : null));
127 if (entry != null) { 119 if (entry != null) {
128 future.complete(entry); 120 future.complete(entry);
......
...@@ -15,15 +15,10 @@ ...@@ -15,15 +15,10 @@
15 */ 15 */
16 package org.onosproject.store.consistent.impl; 16 package org.onosproject.store.consistent.impl;
17 17
18 -import com.google.common.base.Charsets;
19 -import org.onosproject.cluster.NodeId;
20 import org.onosproject.store.service.DistributedQueue; 18 import org.onosproject.store.service.DistributedQueue;
21 import org.onosproject.store.service.DistributedQueueBuilder; 19 import org.onosproject.store.service.DistributedQueueBuilder;
22 import org.onosproject.store.service.Serializer; 20 import org.onosproject.store.service.Serializer;
23 21
24 -import java.util.Set;
25 -import java.util.function.Consumer;
26 -
27 import static com.google.common.base.Preconditions.checkArgument; 22 import static com.google.common.base.Preconditions.checkArgument;
28 import static com.google.common.base.Preconditions.checkState; 23 import static com.google.common.base.Preconditions.checkState;
29 24
...@@ -40,8 +35,7 @@ public class DefaultDistributedQueueBuilder<E> implements DistributedQueueBuilde ...@@ -40,8 +35,7 @@ public class DefaultDistributedQueueBuilder<E> implements DistributedQueueBuilde
40 private final DatabaseManager databaseManager; 35 private final DatabaseManager databaseManager;
41 private boolean metering = true; 36 private boolean metering = true;
42 37
43 - public DefaultDistributedQueueBuilder( 38 + public DefaultDistributedQueueBuilder(DatabaseManager databaseManager) {
44 - DatabaseManager databaseManager) {
45 this.databaseManager = databaseManager; 39 this.databaseManager = databaseManager;
46 } 40 }
47 41
...@@ -78,18 +72,10 @@ public class DefaultDistributedQueueBuilder<E> implements DistributedQueueBuilde ...@@ -78,18 +72,10 @@ public class DefaultDistributedQueueBuilder<E> implements DistributedQueueBuilde
78 @Override 72 @Override
79 public DistributedQueue<E> build() { 73 public DistributedQueue<E> build() {
80 checkState(validInputs()); 74 checkState(validInputs());
81 - Consumer<Set<NodeId>> notifyOthers = nodes -> databaseManager.clusterCommunicator.multicast(name, 75 + return new DefaultDistributedQueue<>(
82 - DatabaseManager.QUEUE_UPDATED_TOPIC,
83 - s -> s.getBytes(Charsets.UTF_8),
84 - nodes);
85 - DefaultDistributedQueue<E> queue = new DefaultDistributedQueue<>(
86 name, 76 name,
87 persistenceEnabled ? databaseManager.partitionedDatabase : databaseManager.inMemoryDatabase, 77 persistenceEnabled ? databaseManager.partitionedDatabase : databaseManager.inMemoryDatabase,
88 serializer, 78 serializer,
89 - databaseManager.localNodeId, 79 + metering);
90 - metering,
91 - notifyOthers);
92 - databaseManager.registerQueue(queue);
93 - return queue;
94 } 80 }
95 } 81 }
......
...@@ -28,7 +28,6 @@ import java.util.concurrent.atomic.AtomicInteger; ...@@ -28,7 +28,6 @@ import java.util.concurrent.atomic.AtomicInteger;
28 import java.util.function.Consumer; 28 import java.util.function.Consumer;
29 import java.util.stream.Collectors; 29 import java.util.stream.Collectors;
30 30
31 -import org.onosproject.cluster.NodeId;
32 import org.onosproject.store.service.DatabaseUpdate; 31 import org.onosproject.store.service.DatabaseUpdate;
33 import org.onosproject.store.service.Transaction; 32 import org.onosproject.store.service.Transaction;
34 import org.onosproject.store.service.Versioned; 33 import org.onosproject.store.service.Versioned;
...@@ -229,15 +228,15 @@ public class PartitionedDatabase implements Database { ...@@ -229,15 +228,15 @@ public class PartitionedDatabase implements Database {
229 } 228 }
230 229
231 @Override 230 @Override
232 - public CompletableFuture<Set<NodeId>> queuePush(String queueName, byte[] entry) { 231 + public CompletableFuture<Void> queuePush(String queueName, byte[] entry) {
233 checkState(isOpen.get(), DB_NOT_OPEN); 232 checkState(isOpen.get(), DB_NOT_OPEN);
234 return partitioner.getPartition(queueName, queueName).queuePush(queueName, entry); 233 return partitioner.getPartition(queueName, queueName).queuePush(queueName, entry);
235 } 234 }
236 235
237 @Override 236 @Override
238 - public CompletableFuture<byte[]> queuePop(String queueName, NodeId nodeId) { 237 + public CompletableFuture<byte[]> queuePop(String queueName) {
239 checkState(isOpen.get(), DB_NOT_OPEN); 238 checkState(isOpen.get(), DB_NOT_OPEN);
240 - return partitioner.getPartition(queueName, queueName).queuePop(queueName, nodeId); 239 + return partitioner.getPartition(queueName, queueName).queuePop(queueName);
241 } 240 }
242 241
243 @Override 242 @Override
......
...@@ -29,7 +29,7 @@ public class StateMachineUpdate { ...@@ -29,7 +29,7 @@ public class StateMachineUpdate {
29 /** 29 /**
30 * Update is for a map. 30 * Update is for a map.
31 */ 31 */
32 - MAP, 32 + MAP_UPDATE,
33 33
34 /** 34 /**
35 * Update is a transaction commit. 35 * Update is a transaction commit.
...@@ -37,7 +37,12 @@ public class StateMachineUpdate { ...@@ -37,7 +37,12 @@ public class StateMachineUpdate {
37 TX_COMMIT, 37 TX_COMMIT,
38 38
39 /** 39 /**
40 - * Update is for a non-map data structure. 40 + * Update is a queue push.
41 + */
42 + QUEUE_PUSH,
43 +
44 + /**
45 + * Update is for some other operation.
41 */ 46 */
42 OTHER 47 OTHER
43 } 48 }
...@@ -55,9 +60,11 @@ public class StateMachineUpdate { ...@@ -55,9 +60,11 @@ public class StateMachineUpdate {
55 public Target target() { 60 public Target target() {
56 // FIXME: This check is brittle 61 // FIXME: This check is brittle
57 if (operationName.contains("mapUpdate")) { 62 if (operationName.contains("mapUpdate")) {
58 - return Target.MAP; 63 + return Target.MAP_UPDATE;
59 } else if (operationName.contains("commit") || operationName.contains("prepareAndCommit")) { 64 } else if (operationName.contains("commit") || operationName.contains("prepareAndCommit")) {
60 return Target.TX_COMMIT; 65 return Target.TX_COMMIT;
66 + } else if (operationName.contains("queuePush")) {
67 + return Target.QUEUE_PUSH;
61 } else { 68 } else {
62 return Target.OTHER; 69 return Target.OTHER;
63 } 70 }
......