Madan Jampani
Committed by Gerrit Code Review

Distributed work queue primitive

Change-Id: Ia8e531e6611ec502399edec376ccc00522e47994
Showing 23 changed files with 1493 additions and 15 deletions
...@@ -15,6 +15,7 @@ ...@@ -15,6 +15,7 @@
15 */ 15 */
16 package org.onosproject.vtnrsc.util; 16 package org.onosproject.vtnrsc.util;
17 17
18 +import org.onosproject.store.service.WorkQueue;
18 import org.onosproject.store.service.EventuallyConsistentMapBuilder; 19 import org.onosproject.store.service.EventuallyConsistentMapBuilder;
19 import org.onosproject.store.service.ConsistentMapBuilder; 20 import org.onosproject.store.service.ConsistentMapBuilder;
20 import org.onosproject.store.service.DistributedSetBuilder; 21 import org.onosproject.store.service.DistributedSetBuilder;
...@@ -22,6 +23,7 @@ import org.onosproject.store.service.DistributedQueueBuilder; ...@@ -22,6 +23,7 @@ import org.onosproject.store.service.DistributedQueueBuilder;
22 import org.onosproject.store.service.AtomicCounterBuilder; 23 import org.onosproject.store.service.AtomicCounterBuilder;
23 import org.onosproject.store.service.AtomicValueBuilder; 24 import org.onosproject.store.service.AtomicValueBuilder;
24 import org.onosproject.store.service.LeaderElectorBuilder; 25 import org.onosproject.store.service.LeaderElectorBuilder;
26 +import org.onosproject.store.service.Serializer;
25 import org.onosproject.store.service.TransactionContextBuilder; 27 import org.onosproject.store.service.TransactionContextBuilder;
26 import org.onosproject.store.service.StorageService; 28 import org.onosproject.store.service.StorageService;
27 29
...@@ -68,4 +70,9 @@ public class VtnStorageServiceAdapter implements StorageService { ...@@ -68,4 +70,9 @@ public class VtnStorageServiceAdapter implements StorageService {
68 public LeaderElectorBuilder leaderElectorBuilder() { 70 public LeaderElectorBuilder leaderElectorBuilder() {
69 return null; 71 return null;
70 } 72 }
73 +
74 + @Override
75 + public <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer) {
76 + return null;
77 + }
71 } 78 }
......
1 +/*
2 + * Copyright 2015-present 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.cli.net;
17 +
18 +import java.util.Map;
19 +
20 +import org.apache.karaf.shell.commands.Command;
21 +import org.onosproject.cli.AbstractShellCommand;
22 +import org.onosproject.store.service.StorageAdminService;
23 +import org.onosproject.store.service.WorkQueueStats;
24 +
25 +import com.fasterxml.jackson.databind.ObjectMapper;
26 +import com.fasterxml.jackson.databind.node.ObjectNode;
27 +
28 +/**
29 + * Command to list stats for all work queues in the system.
30 + */
31 +@Command(scope = "onos", name = "queues",
32 + description = "Lists information about work queues in the system")
33 +public class QueuesListCommand extends AbstractShellCommand {
34 +
35 + private static final String FMT = "name=%s pending=%d inProgress=%d, completed=%d";
36 +
37 + @Override
38 + protected void execute() {
39 + StorageAdminService storageAdminService = get(StorageAdminService.class);
40 + Map<String, WorkQueueStats> queueStats = storageAdminService.getQueueStats();
41 + if (outputJson()) {
42 + ObjectMapper mapper = new ObjectMapper();
43 + ObjectNode jsonQueues = mapper.createObjectNode();
44 + queueStats.forEach((k, v) -> {
45 + ObjectNode jsonStats = jsonQueues.putObject(k);
46 + jsonStats.put("pending", v.totalPending());
47 + jsonStats.put("inProgress", v.totalInProgress());
48 + jsonStats.put("completed", v.totalCompleted());
49 + });
50 + print("%s", jsonQueues);
51 + } else {
52 + queueStats.forEach((name, stats) ->
53 + print(FMT, name, stats.totalPending(), stats.totalInProgress(), stats.totalCompleted()));
54 + }
55 + }
56 +}
...@@ -400,6 +400,9 @@ ...@@ -400,6 +400,9 @@
400 <action class="org.onosproject.cli.net.CountersListCommand"/> 400 <action class="org.onosproject.cli.net.CountersListCommand"/>
401 </command> 401 </command>
402 <command> 402 <command>
403 + <action class="org.onosproject.cli.net.QueuesListCommand"/>
404 + </command>
405 + <command>
403 <action class="org.onosproject.cli.net.TransactionsCommand"/> 406 <action class="org.onosproject.cli.net.TransactionsCommand"/>
404 </command> 407 </command>
405 <command> 408 <command>
......
...@@ -23,6 +23,7 @@ import org.onosproject.store.service.AsyncConsistentMap; ...@@ -23,6 +23,7 @@ import org.onosproject.store.service.AsyncConsistentMap;
23 import org.onosproject.store.service.AsyncDistributedSet; 23 import org.onosproject.store.service.AsyncDistributedSet;
24 import org.onosproject.store.service.AsyncLeaderElector; 24 import org.onosproject.store.service.AsyncLeaderElector;
25 import org.onosproject.store.service.DistributedQueue; 25 import org.onosproject.store.service.DistributedQueue;
26 +import org.onosproject.store.service.WorkQueue;
26 import org.onosproject.store.service.Serializer; 27 import org.onosproject.store.service.Serializer;
27 28
28 /** 29 /**
...@@ -88,6 +89,15 @@ public interface DistributedPrimitiveCreator { ...@@ -88,6 +89,15 @@ public interface DistributedPrimitiveCreator {
88 AsyncLeaderElector newAsyncLeaderElector(String name); 89 AsyncLeaderElector newAsyncLeaderElector(String name);
89 90
90 /** 91 /**
92 + * Creates a new {@code WorkQueue}.
93 + *
94 + * @param name work queue name
95 + * @param serializer serializer
96 + * @return work queue
97 + */
98 + <E> WorkQueue<E> newWorkQueue(String name, Serializer serializer);
99 +
100 + /**
91 * Returns the names of all created {@code AsyncConsistentMap} instances. 101 * Returns the names of all created {@code AsyncConsistentMap} instances.
92 * @return set of {@code AsyncConsistentMap} names 102 * @return set of {@code AsyncConsistentMap} names
93 */ 103 */
...@@ -98,4 +108,10 @@ public interface DistributedPrimitiveCreator { ...@@ -98,4 +108,10 @@ public interface DistributedPrimitiveCreator {
98 * @return set of {@code AsyncAtomicCounter} names 108 * @return set of {@code AsyncAtomicCounter} names
99 */ 109 */
100 Set<String> getAsyncAtomicCounterNames(); 110 Set<String> getAsyncAtomicCounterNames();
111 +
112 + /**
113 + * Returns the names of all created {@code WorkQueue} instances.
114 + * @return set of {@code WorkQueue} names
115 + */
116 + Set<String> getWorkQueueNames();
101 } 117 }
...\ No newline at end of file ...\ No newline at end of file
......
...@@ -52,6 +52,13 @@ public interface StorageAdminService { ...@@ -52,6 +52,13 @@ public interface StorageAdminService {
52 Map<String, Long> getCounters(); 52 Map<String, Long> getCounters();
53 53
54 /** 54 /**
55 + * Returns statistics for all the work queues in the system.
56 + *
57 + * @return mapping from queue name to that queue's stats
58 + */
59 + Map<String, WorkQueueStats> getQueueStats();
60 +
61 + /**
55 * Returns all pending transactions. 62 * Returns all pending transactions.
56 * 63 *
57 * @return collection of pending transaction identifiers. 64 * @return collection of pending transaction identifiers.
......
...@@ -107,4 +107,13 @@ public interface StorageService { ...@@ -107,4 +107,13 @@ public interface StorageService {
107 default AtomicCounter getAtomicCounter(String name) { 107 default AtomicCounter getAtomicCounter(String name) {
108 return getAsyncAtomicCounter(name).asAtomicCounter(); 108 return getAsyncAtomicCounter(name).asAtomicCounter();
109 } 109 }
110 +
111 + /**
112 + * Returns an instance of {@code WorkQueue} with specified name.
113 + * @param name work queue name
114 + * @param serializer serializer
115 + *
116 + * @return WorkQueue instance
117 + */
118 + <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer);
110 } 119 }
......
1 +/*
2 + * Copyright 2016-present 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.service;
17 +
18 +import java.util.function.Function;
19 +
20 +import com.google.common.base.MoreObjects;
21 +
22 +/**
23 + * {@link WorkQueue} task.
24 + *
25 + * @param <E> task payload type.
26 + */
27 +public class Task<E> {
28 + private final E payload;
29 + private final String taskId;
30 +
31 + private Task() {
32 + payload = null;
33 + taskId = null;
34 + }
35 +
36 + /**
37 + * Constructs a new task instance.
38 + * @param taskId task identifier
39 + * @param payload task payload
40 + */
41 + public Task(String taskId, E payload) {
42 + this.taskId = taskId;
43 + this.payload = payload;
44 + }
45 +
46 + /**
47 + * Returns the task identifier.
48 + * @return task id
49 + */
50 + public String taskId() {
51 + return taskId;
52 + }
53 +
54 + /**
55 + * Returns the task payload.
56 + * @return task payload
57 + */
58 + public E payload() {
59 + return payload;
60 + }
61 +
62 + /**
63 + * Maps task from one payload type to another.
64 + * @param mapper type mapper.
65 + * @return mapped task.
66 + */
67 + public <F> Task<F> map(Function<E, F> mapper) {
68 + return new Task<>(taskId, mapper.apply(payload));
69 + }
70 +
71 + @Override
72 + public String toString() {
73 + return MoreObjects.toStringHelper(getClass())
74 + .add("taskId", taskId)
75 + .add("payload", payload)
76 + .toString();
77 + }
78 +}
1 +/*
2 + * Copyright 2016-present 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.service;
17 +
18 +import java.util.Arrays;
19 +import java.util.Collection;
20 +import java.util.concurrent.CompletableFuture;
21 +import java.util.concurrent.Executor;
22 +import java.util.function.Consumer;
23 +
24 +import com.google.common.collect.ImmutableList;
25 +
26 +/**
27 + * Distributed Work Queue primitive.
28 + * <p>
29 + * Work queue serves as a buffer allowing producers to {@link #add(Collection) add} tasks and consumers
30 + * to {@link #take() take} tasks to process.
31 + * <p>
32 + * In the system each task is tracked via its unique task identifier which is returned when a task is taken.
33 + * Work queue guarantees that a task can be taken by only one consumer at a time. Once it finishes processing a
34 + * consumer must invoke the {@link #complete(Collection) complete} method to mark the task(s) as completed.
35 + * Tasks thus completed are removed from the queue. If a consumer unexpectedly terminates before it can complete
36 + * all its tasks are returned back to the queue so that other consumers can pick them up. Since there is a distinct
37 + * possibility that tasks could be processed more than once (under failure conditions), care should be taken to ensure
38 + * task processing logic is idempotent.
39 + *
40 + * @param <E> task payload type.
41 + */
42 +public interface WorkQueue<E> {
43 +
44 + /**
45 + * Adds a collection of tasks to the work queue.
46 + * @param items collection of task items
47 + * @return future that is completed when the operation completes
48 + */
49 + CompletableFuture<Void> addMultiple(Collection<E> items);
50 +
51 + /**
52 + * Picks up multiple tasks from the work queue to work on.
53 + * <p>
54 + * Tasks that are taken remain invisible to other consumers as long as the consumer stays alive.
55 + * If a consumer unexpectedly terminates before {@link #complete(String...) completing} the task,
56 + * the task becomes visible again to other consumers to process.
57 + * @param maxItems maximum number of items to take from the queue. The actual number of tasks returned
58 + * can be at the max this number
59 + * @return future for the tasks. The future can be completed with an empty collection if there are no
60 + * unassigned tasks in the work queue
61 + */
62 + CompletableFuture<Collection<Task<E>>> take(int maxItems);
63 +
64 + /**
65 + * Completes a collection of tasks.
66 + * @param taskIds ids of tasks to complete
67 + * @return future that is completed when the operation completes
68 + */
69 + CompletableFuture<Void> complete(Collection<String> taskIds);
70 +
71 + /**
72 + * Registers a task processing callback to be automatically invoked when new tasks are
73 + * added to the work queue.
74 + * @param taskProcessor task processing callback
75 + * @param parallelism max tasks that can be processed in parallel
76 + * @param executor executor to use for processing the tasks
77 + * @return future that is completed when the operation completes
78 + */
79 + CompletableFuture<Void> registerTaskProcessor(Consumer<E> taskProcessor,
80 + int parallelism,
81 + Executor executor);
82 +
83 + /**
84 + * Stops automatically processing tasks from work queue. This call nullifies the effect of a
85 + * previous {@link #registerTaskProcessor registerTaskProcessor} call.
86 + * @return future that is completed when the operation completes
87 + */
88 + CompletableFuture<Void> stopProcessing();
89 +
90 + /**
91 + * Returns work queue statistics.
92 + * @return future that is completed with work queue stats when the operation completes
93 + */
94 + CompletableFuture<WorkQueueStats> stats();
95 +
96 + /**
97 + * Completes a collection of tasks.
98 + * @param taskIds var arg list of task ids
99 + * @return future that is completed when the operation completes
100 + */
101 + default CompletableFuture<Void> complete(String... taskIds) {
102 + return complete(Arrays.asList(taskIds));
103 + }
104 +
105 + /**
106 + * Adds a single task to the work queue.
107 + * @param item task item
108 + * @return future that is completed when the operation completes
109 + */
110 + default CompletableFuture<Void> addOne(E item) {
111 + return addMultiple(ImmutableList.of(item));
112 + }
113 +
114 + /**
115 + * Picks up a single task from the work queue to work on.
116 + * <p>
117 + * Tasks that are taken remain invisible to other consumers as long as the consumer stays alive.
118 + * If a consumer unexpectedly terminates before {@link #complete(String...) completing} the task,
119 + * the task becomes visible again to other consumers to process.
120 + * @return future for the task. The future can be completed with null, if there are no
121 + * unassigned tasks in the work queue
122 + */
123 + default CompletableFuture<Task<E>> take() {
124 + return this.take(1).thenApply(tasks -> tasks.isEmpty() ? null : tasks.iterator().next());
125 + }
126 +}
...\ No newline at end of file ...\ No newline at end of file
1 +/*
2 + * Copyright 2016-present 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.service;
17 +
18 +import com.google.common.base.MoreObjects;
19 +
20 +/**
21 + * Statistics for a {@link WorkQueue}.
22 + */
23 +public final class WorkQueueStats {
24 +
25 + private long totalPending;
26 + private long totalInProgress;
27 + private long totalCompleted;
28 +
29 + /**
30 + * Returns a {@code WorkQueueStats} builder.
31 + * @return builder
32 + */
33 + public static Builder builder() {
34 + return new Builder();
35 + }
36 +
37 + private WorkQueueStats() {
38 + }
39 +
40 + public static class Builder {
41 +
42 + WorkQueueStats workQueueStats = new WorkQueueStats();
43 +
44 + public Builder withTotalPending(long value) {
45 + workQueueStats.totalPending = value;
46 + return this;
47 + }
48 +
49 + public Builder withTotalInProgress(long value) {
50 + workQueueStats.totalInProgress = value;
51 + return this;
52 + }
53 +
54 + public Builder withTotalCompleted(long value) {
55 + workQueueStats.totalCompleted = value;
56 + return this;
57 + }
58 +
59 + public WorkQueueStats build() {
60 + return workQueueStats;
61 + }
62 + }
63 +
64 + /**
65 + * Returns the total pending tasks. These are the tasks that are added but not yet picked up.
66 + * @return total pending tasks.
67 + */
68 + public long totalPending() {
69 + return this.totalPending;
70 + }
71 +
72 + /**
73 + * Returns the total in progress tasks. These are the tasks that are currently being worked on.
74 + * @return total in progress tasks.
75 + */
76 + public long totalInProgress() {
77 + return this.totalInProgress;
78 + }
79 +
80 + /**
81 + * Returns the total completed tasks.
82 + * @return total completed tasks.
83 + */
84 + public long totalCompleted() {
85 + return this.totalCompleted;
86 + }
87 +
88 + @Override
89 + public String toString() {
90 + return MoreObjects.toStringHelper(getClass())
91 + .add("totalPending", totalPending)
92 + .add("totalInProgress", totalInProgress)
93 + .add("totalCompleted", totalCompleted)
94 + .toString();
95 + }
96 +}
...@@ -58,4 +58,9 @@ public class StorageServiceAdapter implements StorageService { ...@@ -58,4 +58,9 @@ public class StorageServiceAdapter implements StorageService {
58 public LeaderElectorBuilder leaderElectorBuilder() { 58 public LeaderElectorBuilder leaderElectorBuilder() {
59 return null; 59 return null;
60 } 60 }
61 +
62 + @Override
63 + public <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer) {
64 + return null;
65 + }
61 } 66 }
......
...@@ -33,6 +33,8 @@ import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapComman ...@@ -33,6 +33,8 @@ import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapComman
33 import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapFactory; 33 import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapFactory;
34 import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands; 34 import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands;
35 import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorFactory; 35 import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorFactory;
36 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands;
37 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueFactory;
36 import org.onosproject.store.primitives.resources.impl.CommitResult; 38 import org.onosproject.store.primitives.resources.impl.CommitResult;
37 import org.onosproject.store.primitives.resources.impl.MapEntryUpdateResult; 39 import org.onosproject.store.primitives.resources.impl.MapEntryUpdateResult;
38 import org.onosproject.store.primitives.resources.impl.PrepareResult; 40 import org.onosproject.store.primitives.resources.impl.PrepareResult;
...@@ -40,8 +42,11 @@ import org.onosproject.store.primitives.resources.impl.RollbackResult; ...@@ -40,8 +42,11 @@ import org.onosproject.store.primitives.resources.impl.RollbackResult;
40 import org.onosproject.store.serializers.KryoNamespaces; 42 import org.onosproject.store.serializers.KryoNamespaces;
41 import org.onosproject.store.service.MapEvent; 43 import org.onosproject.store.service.MapEvent;
42 import org.onosproject.store.service.MapTransaction; 44 import org.onosproject.store.service.MapTransaction;
45 +import org.onosproject.store.service.Task;
43 import org.onosproject.store.service.Versioned; 46 import org.onosproject.store.service.Versioned;
47 +import org.onosproject.store.service.WorkQueueStats;
44 48
49 +import com.google.common.collect.ImmutableList;
45 import com.google.common.collect.Maps; 50 import com.google.common.collect.Maps;
46 51
47 /** 52 /**
...@@ -81,15 +86,20 @@ public final class CatalystSerializers { ...@@ -81,15 +86,20 @@ public final class CatalystSerializers {
81 serializer.register(MapTransaction.class, factory); 86 serializer.register(MapTransaction.class, factory);
82 serializer.register(Versioned.class, factory); 87 serializer.register(Versioned.class, factory);
83 serializer.register(MapEvent.class, factory); 88 serializer.register(MapEvent.class, factory);
89 + serializer.register(Task.class, factory);
90 + serializer.register(WorkQueueStats.class, factory);
84 serializer.register(Maps.immutableEntry("a", "b").getClass(), factory); 91 serializer.register(Maps.immutableEntry("a", "b").getClass(), factory);
92 + serializer.register(ImmutableList.of().getClass(), factory);
85 93
86 serializer.resolve(new LongCommands.TypeResolver()); 94 serializer.resolve(new LongCommands.TypeResolver());
87 serializer.resolve(new AtomixConsistentMapCommands.TypeResolver()); 95 serializer.resolve(new AtomixConsistentMapCommands.TypeResolver());
88 serializer.resolve(new AtomixLeaderElectorCommands.TypeResolver()); 96 serializer.resolve(new AtomixLeaderElectorCommands.TypeResolver());
97 + serializer.resolve(new AtomixWorkQueueCommands.TypeResolver());
89 serializer.resolve(new ResourceManagerTypeResolver()); 98 serializer.resolve(new ResourceManagerTypeResolver());
90 99
91 serializer.registerClassLoader(AtomixConsistentMapFactory.class) 100 serializer.registerClassLoader(AtomixConsistentMapFactory.class)
92 - .registerClassLoader(AtomixLeaderElectorFactory.class); 101 + .registerClassLoader(AtomixLeaderElectorFactory.class)
102 + .registerClassLoader(AtomixWorkQueueFactory.class);
93 103
94 return serializer; 104 return serializer;
95 } 105 }
......
...@@ -54,20 +54,19 @@ public class DefaultCatalystTypeSerializerFactory implements TypeSerializerFacto ...@@ -54,20 +54,19 @@ public class DefaultCatalystTypeSerializerFactory implements TypeSerializerFacto
54 } 54 }
55 55
56 @Override 56 @Override
57 - public void write(T object, BufferOutput buffer, 57 + public void write(T object, BufferOutput buffer, io.atomix.catalyst.serializer.Serializer serializer) {
58 - io.atomix.catalyst.serializer.Serializer serializer) {
59 try { 58 try {
60 byte[] payload = this.serializer.encode(object); 59 byte[] payload = this.serializer.encode(object);
61 buffer.writeInt(payload.length); 60 buffer.writeInt(payload.length);
62 buffer.write(payload); 61 buffer.write(payload);
63 } catch (Exception e) { 62 } catch (Exception e) {
64 log.warn("Failed to serialize {}", object, e); 63 log.warn("Failed to serialize {}", object, e);
64 + throw Throwables.propagate(e);
65 } 65 }
66 } 66 }
67 67
68 @Override 68 @Override
69 - public T read(Class<T> type, BufferInput buffer, 69 + public T read(Class<T> type, BufferInput buffer, io.atomix.catalyst.serializer.Serializer serializer) {
70 - io.atomix.catalyst.serializer.Serializer serializer) {
71 int size = buffer.readInt(); 70 int size = buffer.readInt();
72 try { 71 try {
73 byte[] payload = new byte[size]; 72 byte[] payload = new byte[size];
...@@ -75,8 +74,7 @@ public class DefaultCatalystTypeSerializerFactory implements TypeSerializerFacto ...@@ -75,8 +74,7 @@ public class DefaultCatalystTypeSerializerFactory implements TypeSerializerFacto
75 return this.serializer.decode(payload); 74 return this.serializer.decode(payload);
76 } catch (Exception e) { 75 } catch (Exception e) {
77 log.warn("Failed to deserialize as type {}. Payload size: {}", type, size, e); 76 log.warn("Failed to deserialize as type {}. Payload size: {}", type, size, e);
78 - Throwables.propagate(e); 77 + throw Throwables.propagate(e);
79 - return null;
80 } 78 }
81 } 79 }
82 } 80 }
......
1 +package org.onosproject.store.primitives.impl;
2 +
3 +import java.util.ArrayList;
4 +import java.util.Collection;
5 +import java.util.concurrent.CompletableFuture;
6 +import java.util.concurrent.Executor;
7 +import java.util.function.Consumer;
8 +import java.util.stream.Collectors;
9 +
10 +import org.onosproject.store.service.WorkQueue;
11 +import org.onosproject.store.service.Serializer;
12 +import org.onosproject.store.service.Task;
13 +import org.onosproject.store.service.WorkQueueStats;
14 +
15 +import com.google.common.collect.Collections2;
16 +
17 +/**
18 + * Default implementation of {@link WorkQueue}.
19 + *
20 + * @param <E> task payload type.
21 + */
22 +public class DefaultDistributedWorkQueue<E> implements WorkQueue<E> {
23 +
24 + private final WorkQueue<byte[]> backingQueue;
25 + private final Serializer serializer;
26 +
27 + public DefaultDistributedWorkQueue(WorkQueue<byte[]> backingQueue, Serializer serializer) {
28 + this.backingQueue = backingQueue;
29 + this.serializer = serializer;
30 + }
31 +
32 + @Override
33 + public CompletableFuture<Void> addMultiple(Collection<E> items) {
34 + return backingQueue.addMultiple(items.stream()
35 + .map(serializer::encode)
36 + .collect(Collectors.toCollection(ArrayList::new)));
37 + }
38 +
39 + @Override
40 + public CompletableFuture<Collection<Task<E>>> take(int maxTasks) {
41 + return backingQueue.take(maxTasks)
42 + .thenApply(tasks -> Collections2.transform(tasks, task -> task.<E>map(serializer::decode)));
43 + }
44 +
45 + @Override
46 + public CompletableFuture<Void> complete(Collection<String> ids) {
47 + return backingQueue.complete(ids);
48 + }
49 +
50 + @Override
51 + public CompletableFuture<WorkQueueStats> stats() {
52 + return backingQueue.stats();
53 + }
54 +
55 + @Override
56 + public CompletableFuture<Void> registerTaskProcessor(Consumer<E> callback,
57 + int parallelism,
58 + Executor executor) {
59 + Consumer<byte[]> backingQueueCallback = payload -> callback.accept(serializer.decode(payload));
60 + return backingQueue.registerTaskProcessor(backingQueueCallback, parallelism, executor);
61 + }
62 +
63 + @Override
64 + public CompletableFuture<Void> stopProcessing() {
65 + return backingQueue.stopProcessing();
66 + }
67 +}
...@@ -30,6 +30,7 @@ import org.onosproject.store.service.AsyncConsistentMap; ...@@ -30,6 +30,7 @@ import org.onosproject.store.service.AsyncConsistentMap;
30 import org.onosproject.store.service.AsyncDistributedSet; 30 import org.onosproject.store.service.AsyncDistributedSet;
31 import org.onosproject.store.service.AsyncLeaderElector; 31 import org.onosproject.store.service.AsyncLeaderElector;
32 import org.onosproject.store.service.DistributedQueue; 32 import org.onosproject.store.service.DistributedQueue;
33 +import org.onosproject.store.service.WorkQueue;
33 import org.onosproject.store.service.Serializer; 34 import org.onosproject.store.service.Serializer;
34 35
35 import com.google.common.base.Charsets; 36 import com.google.common.base.Charsets;
...@@ -101,6 +102,11 @@ public class FederatedDistributedPrimitiveCreator implements DistributedPrimitiv ...@@ -101,6 +102,11 @@ public class FederatedDistributedPrimitiveCreator implements DistributedPrimitiv
101 } 102 }
102 103
103 @Override 104 @Override
105 + public <E> WorkQueue<E> newWorkQueue(String name, Serializer serializer) {
106 + return getCreator(name).newWorkQueue(name, serializer);
107 + }
108 +
109 + @Override
104 public Set<String> getAsyncConsistentMapNames() { 110 public Set<String> getAsyncConsistentMapNames() {
105 return members.values() 111 return members.values()
106 .stream() 112 .stream()
...@@ -118,6 +124,15 @@ public class FederatedDistributedPrimitiveCreator implements DistributedPrimitiv ...@@ -118,6 +124,15 @@ public class FederatedDistributedPrimitiveCreator implements DistributedPrimitiv
118 .orElse(ImmutableSet.of()); 124 .orElse(ImmutableSet.of());
119 } 125 }
120 126
127 + @Override
128 + public Set<String> getWorkQueueNames() {
129 + return members.values()
130 + .stream()
131 + .map(DistributedPrimitiveCreator::getWorkQueueNames)
132 + .reduce(Sets::union)
133 + .orElse(ImmutableSet.of());
134 + }
135 +
121 /** 136 /**
122 * Returns the {@code DistributedPrimitiveCreator} to use for hosting a primitive. 137 * Returns the {@code DistributedPrimitiveCreator} to use for hosting a primitive.
123 * @param name primitive name 138 * @param name primitive name
......
...@@ -46,6 +46,7 @@ import org.onosproject.store.service.ConsistentMap; ...@@ -46,6 +46,7 @@ import org.onosproject.store.service.ConsistentMap;
46 import org.onosproject.store.service.ConsistentMapBuilder; 46 import org.onosproject.store.service.ConsistentMapBuilder;
47 import org.onosproject.store.service.DistributedQueueBuilder; 47 import org.onosproject.store.service.DistributedQueueBuilder;
48 import org.onosproject.store.service.DistributedSetBuilder; 48 import org.onosproject.store.service.DistributedSetBuilder;
49 +import org.onosproject.store.service.WorkQueue;
49 import org.onosproject.store.service.EventuallyConsistentMapBuilder; 50 import org.onosproject.store.service.EventuallyConsistentMapBuilder;
50 import org.onosproject.store.service.LeaderElectorBuilder; 51 import org.onosproject.store.service.LeaderElectorBuilder;
51 import org.onosproject.store.service.MapInfo; 52 import org.onosproject.store.service.MapInfo;
...@@ -54,6 +55,7 @@ import org.onosproject.store.service.Serializer; ...@@ -54,6 +55,7 @@ import org.onosproject.store.service.Serializer;
54 import org.onosproject.store.service.StorageAdminService; 55 import org.onosproject.store.service.StorageAdminService;
55 import org.onosproject.store.service.StorageService; 56 import org.onosproject.store.service.StorageService;
56 import org.onosproject.store.service.TransactionContextBuilder; 57 import org.onosproject.store.service.TransactionContextBuilder;
58 +import org.onosproject.store.service.WorkQueueStats;
57 import org.slf4j.Logger; 59 import org.slf4j.Logger;
58 60
59 import com.google.common.collect.Maps; 61 import com.google.common.collect.Maps;
...@@ -171,6 +173,12 @@ public class StorageManager implements StorageService, StorageAdminService { ...@@ -171,6 +173,12 @@ public class StorageManager implements StorageService, StorageAdminService {
171 } 173 }
172 174
173 @Override 175 @Override
176 + public <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer) {
177 + checkPermission(STORAGE_WRITE);
178 + return federatedPrimitiveCreator.newWorkQueue(name, serializer);
179 + }
180 +
181 + @Override
174 public List<MapInfo> getMapInfo() { 182 public List<MapInfo> getMapInfo() {
175 return listMapInfo(federatedPrimitiveCreator); 183 return listMapInfo(federatedPrimitiveCreator);
176 } 184 }
...@@ -185,6 +193,18 @@ public class StorageManager implements StorageService, StorageAdminService { ...@@ -185,6 +193,18 @@ public class StorageManager implements StorageService, StorageAdminService {
185 } 193 }
186 194
187 @Override 195 @Override
196 + public Map<String, WorkQueueStats> getQueueStats() {
197 + Map<String, WorkQueueStats> workQueueStats = Maps.newConcurrentMap();
198 + federatedPrimitiveCreator.getWorkQueueNames()
199 + .forEach(name -> workQueueStats.put(name,
200 + federatedPrimitiveCreator.newWorkQueue(name,
201 + Serializer.using(KryoNamespaces.BASIC))
202 + .stats()
203 + .join()));
204 + return workQueueStats;
205 + }
206 +
207 + @Override
188 public List<PartitionInfo> getPartitionInfo() { 208 public List<PartitionInfo> getPartitionInfo() {
189 return partitionAdminService.partitionInfo(); 209 return partitionAdminService.partitionInfo();
190 } 210 }
......
...@@ -41,6 +41,7 @@ import org.onosproject.store.primitives.DistributedPrimitiveCreator; ...@@ -41,6 +41,7 @@ import org.onosproject.store.primitives.DistributedPrimitiveCreator;
41 import org.onosproject.store.primitives.resources.impl.AtomixConsistentMap; 41 import org.onosproject.store.primitives.resources.impl.AtomixConsistentMap;
42 import org.onosproject.store.primitives.resources.impl.AtomixCounter; 42 import org.onosproject.store.primitives.resources.impl.AtomixCounter;
43 import org.onosproject.store.primitives.resources.impl.AtomixLeaderElector; 43 import org.onosproject.store.primitives.resources.impl.AtomixLeaderElector;
44 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueue;
44 import org.onosproject.store.serializers.KryoNamespaces; 45 import org.onosproject.store.serializers.KryoNamespaces;
45 import org.onosproject.store.service.AsyncAtomicCounter; 46 import org.onosproject.store.service.AsyncAtomicCounter;
46 import org.onosproject.store.service.AsyncAtomicValue; 47 import org.onosproject.store.service.AsyncAtomicValue;
...@@ -49,6 +50,7 @@ import org.onosproject.store.service.AsyncDistributedSet; ...@@ -49,6 +50,7 @@ import org.onosproject.store.service.AsyncDistributedSet;
49 import org.onosproject.store.service.AsyncLeaderElector; 50 import org.onosproject.store.service.AsyncLeaderElector;
50 import org.onosproject.store.service.DistributedPrimitive.Status; 51 import org.onosproject.store.service.DistributedPrimitive.Status;
51 import org.onosproject.store.service.DistributedQueue; 52 import org.onosproject.store.service.DistributedQueue;
53 +import org.onosproject.store.service.WorkQueue;
52 import org.onosproject.store.service.PartitionClientInfo; 54 import org.onosproject.store.service.PartitionClientInfo;
53 import org.onosproject.store.service.Serializer; 55 import org.onosproject.store.service.Serializer;
54 import org.slf4j.Logger; 56 import org.slf4j.Logger;
...@@ -159,11 +161,16 @@ public class StoragePartitionClient implements DistributedPrimitiveCreator, Mana ...@@ -159,11 +161,16 @@ public class StoragePartitionClient implements DistributedPrimitiveCreator, Mana
159 161
160 @Override 162 @Override
161 public <E> DistributedQueue<E> newDistributedQueue(String name, Serializer serializer) { 163 public <E> DistributedQueue<E> newDistributedQueue(String name, Serializer serializer) {
162 - // TODO: Implement
163 throw new UnsupportedOperationException(); 164 throw new UnsupportedOperationException();
164 } 165 }
165 166
166 @Override 167 @Override
168 + public <E> WorkQueue<E> newWorkQueue(String name, Serializer serializer) {
169 + AtomixWorkQueue workQueue = client.getResource(name, AtomixWorkQueue.class).join();
170 + return new DefaultDistributedWorkQueue<>(workQueue, serializer);
171 + }
172 +
173 + @Override
167 public AsyncLeaderElector newAsyncLeaderElector(String name) { 174 public AsyncLeaderElector newAsyncLeaderElector(String name) {
168 AtomixLeaderElector leaderElector = client.getResource(name, AtomixLeaderElector.class) 175 AtomixLeaderElector leaderElector = client.getResource(name, AtomixLeaderElector.class)
169 .thenCompose(AtomixLeaderElector::setupCache) 176 .thenCompose(AtomixLeaderElector::setupCache)
...@@ -187,6 +194,11 @@ public class StoragePartitionClient implements DistributedPrimitiveCreator, Mana ...@@ -187,6 +194,11 @@ public class StoragePartitionClient implements DistributedPrimitiveCreator, Mana
187 } 194 }
188 195
189 @Override 196 @Override
197 + public Set<String> getWorkQueueNames() {
198 + return client.keys(AtomixWorkQueue.class).join();
199 + }
200 +
201 + @Override
190 public boolean isOpen() { 202 public boolean isOpen() {
191 return resourceClient.client().state() != State.CLOSED; 203 return resourceClient.client().state() != State.CLOSED;
192 } 204 }
......
1 +/*
2 + * Copyright 2016-present Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import static org.slf4j.LoggerFactory.getLogger;
19 +
20 +import java.util.Collection;
21 +import java.util.List;
22 +import java.util.Properties;
23 +import java.util.Timer;
24 +import java.util.concurrent.CompletableFuture;
25 +import java.util.concurrent.Executor;
26 +import java.util.concurrent.ExecutorService;
27 +import java.util.concurrent.Executors;
28 +import java.util.concurrent.atomic.AtomicBoolean;
29 +import java.util.concurrent.atomic.AtomicInteger;
30 +import java.util.concurrent.atomic.AtomicReference;
31 +import java.util.function.Consumer;
32 +
33 +import org.onlab.util.AbstractAccumulator;
34 +import org.onlab.util.Accumulator;
35 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Add;
36 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Complete;
37 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Register;
38 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Stats;
39 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Take;
40 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Unregister;
41 +import org.onosproject.store.service.WorkQueue;
42 +import org.onosproject.store.service.Task;
43 +import org.onosproject.store.service.WorkQueueStats;
44 +import org.slf4j.Logger;
45 +
46 +import com.google.common.collect.ImmutableList;
47 +
48 +import io.atomix.copycat.client.CopycatClient;
49 +import io.atomix.resource.AbstractResource;
50 +import io.atomix.resource.ResourceTypeInfo;
51 +
52 +/**
53 + * Distributed resource providing the {@link WorkQueue} primitive.
54 + */
55 +@ResourceTypeInfo(id = -154, factory = AtomixWorkQueueFactory.class)
56 +public class AtomixWorkQueue extends AbstractResource<AtomixWorkQueue>
57 + implements WorkQueue<byte[]> {
58 +
59 + private final Logger log = getLogger(getClass());
60 + public static final String TASK_AVAILABLE = "task-available";
61 + private final ExecutorService executor = Executors.newSingleThreadExecutor();
62 + private final AtomicReference<TaskProcessor> taskProcessor = new AtomicReference<>();
63 + private final Timer timer = new Timer("atomix-work-queue-completer");
64 + private final AtomicBoolean isRegistered = new AtomicBoolean(false);
65 +
66 + protected AtomixWorkQueue(CopycatClient client, Properties options) {
67 + super(client, options);
68 + }
69 +
70 + @Override
71 + public CompletableFuture<AtomixWorkQueue> open() {
72 + return super.open().thenApply(result -> {
73 + client.onStateChange(state -> {
74 + if (state == CopycatClient.State.CONNECTED && isRegistered.get()) {
75 + client.submit(new Register());
76 + }
77 + });
78 + client.onEvent(TASK_AVAILABLE, this::resumeWork);
79 + return result;
80 + });
81 + }
82 +
83 + @Override
84 + public CompletableFuture<Void> addMultiple(Collection<byte[]> items) {
85 + if (items.isEmpty()) {
86 + return CompletableFuture.completedFuture(null);
87 + }
88 + return client.submit(new Add(items));
89 + }
90 +
91 + @Override
92 + public CompletableFuture<Collection<Task<byte[]>>> take(int maxTasks) {
93 + if (maxTasks <= 0) {
94 + return CompletableFuture.completedFuture(ImmutableList.of());
95 + }
96 + return client.submit(new Take(maxTasks));
97 + }
98 +
99 + @Override
100 + public CompletableFuture<Void> complete(Collection<String> taskIds) {
101 + if (taskIds.isEmpty()) {
102 + return CompletableFuture.completedFuture(null);
103 + }
104 + return client.submit(new Complete(taskIds));
105 + }
106 +
107 + @Override
108 + public CompletableFuture<Void> registerTaskProcessor(Consumer<byte[]> callback,
109 + int parallelism,
110 + Executor executor) {
111 + Accumulator<String> completedTaskAccumulator =
112 + new CompletedTaskAccumulator(timer, 50, 50); // TODO: make configurable
113 + taskProcessor.set(new TaskProcessor(callback,
114 + parallelism,
115 + executor,
116 + completedTaskAccumulator));
117 + return register().thenCompose(v -> take(parallelism))
118 + .thenAccept(taskProcessor.get()::accept);
119 + }
120 +
121 + @Override
122 + public CompletableFuture<Void> stopProcessing() {
123 + return unregister();
124 + }
125 +
126 + @Override
127 + public CompletableFuture<WorkQueueStats> stats() {
128 + return client.submit(new Stats());
129 + }
130 +
131 + private void resumeWork() {
132 + TaskProcessor activeProcessor = taskProcessor.get();
133 + if (activeProcessor == null) {
134 + return;
135 + }
136 + this.take(activeProcessor.headRoom())
137 + .whenCompleteAsync((tasks, e) -> activeProcessor.accept(tasks), executor);
138 + }
139 +
140 + private CompletableFuture<Void> register() {
141 + return client.submit(new Register()).thenRun(() -> isRegistered.set(true));
142 + }
143 +
144 + private CompletableFuture<Void> unregister() {
145 + return client.submit(new Unregister()).thenRun(() -> isRegistered.set(false));
146 + }
147 +
148 + // TaskId accumulator for paced triggering of task completion calls.
149 + private class CompletedTaskAccumulator extends AbstractAccumulator<String> {
150 + CompletedTaskAccumulator(Timer timer, int maxTasksToBatch, int maxBatchMillis) {
151 + super(timer, maxTasksToBatch, maxBatchMillis, Integer.MAX_VALUE);
152 + }
153 +
154 + @Override
155 + public void processItems(List<String> items) {
156 + complete(items);
157 + }
158 + }
159 +
160 + private class TaskProcessor implements Consumer<Collection<Task<byte[]>>> {
161 +
162 + private final AtomicInteger headRoom;
163 + private final Consumer<byte[]> backingConsumer;
164 + private final Executor executor;
165 + private final Accumulator<String> taskCompleter;
166 +
167 + public TaskProcessor(Consumer<byte[]> backingConsumer,
168 + int parallelism,
169 + Executor executor,
170 + Accumulator<String> taskCompleter) {
171 + this.backingConsumer = backingConsumer;
172 + this.headRoom = new AtomicInteger(parallelism);
173 + this.executor = executor;
174 + this.taskCompleter = taskCompleter;
175 + }
176 +
177 + public int headRoom() {
178 + return headRoom.get();
179 + }
180 +
181 + @Override
182 + public void accept(Collection<Task<byte[]>> tasks) {
183 + if (tasks == null) {
184 + return;
185 + }
186 + headRoom.addAndGet(-1 * tasks.size());
187 + tasks.forEach(task ->
188 + executor.execute(() -> {
189 + try {
190 + backingConsumer.accept(task.payload());
191 + taskCompleter.add(task.taskId());
192 + } catch (Exception e) {
193 + log.debug("Task execution failed", e);
194 + } finally {
195 + headRoom.incrementAndGet();
196 + resumeWork();
197 + }
198 + }));
199 + }
200 + }
201 +}
1 +/*
2 + * Copyright 2016-present 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.ArrayList;
19 +import java.util.Collection;
20 +import java.util.stream.Collectors;
21 +import java.util.stream.IntStream;
22 +
23 +import org.onosproject.store.service.Task;
24 +import org.onosproject.store.service.WorkQueueStats;
25 +
26 +import com.google.common.base.MoreObjects;
27 +
28 +import io.atomix.catalyst.buffer.BufferInput;
29 +import io.atomix.catalyst.buffer.BufferOutput;
30 +import io.atomix.catalyst.serializer.CatalystSerializable;
31 +import io.atomix.catalyst.serializer.SerializableTypeResolver;
32 +import io.atomix.catalyst.serializer.Serializer;
33 +import io.atomix.catalyst.serializer.SerializerRegistry;
34 +import io.atomix.copycat.Command;
35 +
36 +/**
37 + * {@link AtomixWorkQueue} resource state machine operations.
38 + */
39 +public final class AtomixWorkQueueCommands {
40 +
41 + private AtomixWorkQueueCommands() {
42 + }
43 +
44 + /**
45 + * Command to add a collection of tasks to the queue.
46 + */
47 + @SuppressWarnings("serial")
48 + public static class Add implements Command<Void>, CatalystSerializable {
49 +
50 + private Collection<byte[]> items;
51 +
52 + private Add() {
53 + }
54 +
55 + public Add(Collection<byte[]> items) {
56 + this.items = items;
57 + }
58 +
59 + @Override
60 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
61 + buffer.writeInt(items.size());
62 + items.forEach(task -> serializer.writeObject(task, buffer));
63 + }
64 +
65 + @Override
66 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
67 + items = IntStream.range(0, buffer.readInt())
68 + .mapToObj(i -> serializer.<byte[]>readObject(buffer))
69 + .collect(Collectors.toCollection(ArrayList::new));
70 + }
71 +
72 + public Collection<byte[]> items() {
73 + return items;
74 + }
75 +
76 + @Override
77 + public String toString() {
78 + return MoreObjects.toStringHelper(getClass())
79 + .add("items", items)
80 + .toString();
81 + }
82 + }
83 +
84 + /**
85 + * Command to take a task from the queue.
86 + */
87 + @SuppressWarnings("serial")
88 + public static class Take implements Command<Collection<Task<byte[]>>>, CatalystSerializable {
89 +
90 + private int maxTasks;
91 +
92 + private Take() {
93 + }
94 +
95 + public Take(int maxTasks) {
96 + this.maxTasks = maxTasks;
97 + }
98 +
99 + @Override
100 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
101 + buffer.writeInt(maxTasks);
102 + }
103 +
104 + @Override
105 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
106 + maxTasks = buffer.readInt();
107 + }
108 +
109 + public int maxTasks() {
110 + return maxTasks;
111 + }
112 +
113 + @Override
114 + public String toString() {
115 + return MoreObjects.toStringHelper(getClass())
116 + .add("maxTasks", maxTasks)
117 + .toString();
118 + }
119 + }
120 +
121 + @SuppressWarnings("serial")
122 + public static class Stats implements Command<WorkQueueStats>, CatalystSerializable {
123 +
124 + @Override
125 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
126 + }
127 +
128 + @Override
129 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
130 + }
131 +
132 + @Override
133 + public String toString() {
134 + return MoreObjects.toStringHelper(getClass())
135 + .toString();
136 + }
137 + }
138 +
139 +
140 +
141 + @SuppressWarnings("serial")
142 + public static class Register implements Command<Void>, CatalystSerializable {
143 +
144 + @Override
145 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
146 + }
147 +
148 + @Override
149 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
150 + }
151 +
152 + @Override
153 + public String toString() {
154 + return MoreObjects.toStringHelper(getClass())
155 + .toString();
156 + }
157 + }
158 +
159 + @SuppressWarnings("serial")
160 + public static class Unregister implements Command<Void>, CatalystSerializable {
161 +
162 + @Override
163 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
164 + }
165 +
166 + @Override
167 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
168 + }
169 +
170 + @Override
171 + public String toString() {
172 + return MoreObjects.toStringHelper(getClass())
173 + .toString();
174 + }
175 + }
176 +
177 + @SuppressWarnings("serial")
178 + public static class Complete implements Command<Void>, CatalystSerializable {
179 + private Collection<String> taskIds;
180 +
181 + private Complete() {
182 + }
183 +
184 + public Complete(Collection<String> taskIds) {
185 + this.taskIds = taskIds;
186 + }
187 +
188 + @Override
189 + public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
190 + serializer.writeObject(taskIds, buffer);
191 + }
192 +
193 + @Override
194 + public void readObject(BufferInput<?> buffer, Serializer serializer) {
195 + taskIds = serializer.readObject(buffer);
196 + }
197 +
198 + public Collection<String> taskIds() {
199 + return taskIds;
200 + }
201 +
202 + @Override
203 + public String toString() {
204 + return MoreObjects.toStringHelper(getClass())
205 + .add("taskIds", taskIds)
206 + .toString();
207 + }
208 + }
209 +
210 + /**
211 + * Work queue command type resolver.
212 + */
213 + public static class TypeResolver implements SerializableTypeResolver {
214 + @Override
215 + public void resolve(SerializerRegistry registry) {
216 + registry.register(Register.class, -960);
217 + registry.register(Unregister.class, -961);
218 + registry.register(Take.class, -962);
219 + registry.register(Add.class, -963);
220 + registry.register(Complete.class, -964);
221 + registry.register(Stats.class, -965);
222 + }
223 + }
224 +}
1 +/*
2 + * Copyright 2016-present 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.serializer.SerializableTypeResolver;
19 +import io.atomix.copycat.client.CopycatClient;
20 +import io.atomix.resource.ResourceFactory;
21 +import io.atomix.resource.ResourceStateMachine;
22 +
23 +import java.util.Properties;
24 +
25 +/**
26 + * {@link AtomixWorkQueue} resource factory.
27 + */
28 +public class AtomixWorkQueueFactory implements ResourceFactory<AtomixWorkQueue> {
29 +
30 + @Override
31 + public SerializableTypeResolver createSerializableTypeResolver() {
32 + return new AtomixWorkQueueCommands.TypeResolver();
33 + }
34 +
35 + @Override
36 + public ResourceStateMachine createStateMachine(Properties config) {
37 + return new AtomixWorkQueueState(config);
38 + }
39 +
40 + @Override
41 + public AtomixWorkQueue createInstance(CopycatClient client, Properties properties) {
42 + return new AtomixWorkQueue(client, properties);
43 + }
44 +}
...\ No newline at end of file ...\ No newline at end of file
1 +/*
2 + * Copyright 2016-present Open Networking Laboratory
3 + *
4 + * Licensed under the Apache License, Version 2.0 (the "License");
5 + * you may not use this file except in compliance with the License.
6 + * You may obtain a copy of the License at
7 + *
8 + * http://www.apache.org/licenses/LICENSE-2.0
9 + *
10 + * Unless required by applicable law or agreed to in writing, software
11 + * distributed under the License is distributed on an "AS IS" BASIS,
12 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 + * See the License for the specific language governing permissions and
14 + * limitations under the License.
15 + */
16 +package org.onosproject.store.primitives.resources.impl;
17 +
18 +import static org.slf4j.LoggerFactory.getLogger;
19 +
20 +import java.util.ArrayList;
21 +import java.util.Collection;
22 +import java.util.Iterator;
23 +import java.util.Map;
24 +import java.util.Properties;
25 +import java.util.Queue;
26 +import java.util.concurrent.atomic.AtomicInteger;
27 +import java.util.concurrent.atomic.AtomicLong;
28 +import java.util.function.Consumer;
29 +import java.util.stream.Collectors;
30 +import java.util.stream.IntStream;
31 +
32 +import org.onlab.util.CountDownCompleter;
33 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Add;
34 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Complete;
35 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Register;
36 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Stats;
37 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Take;
38 +import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Unregister;
39 +import org.onosproject.store.service.Task;
40 +import org.onosproject.store.service.WorkQueueStats;
41 +import org.slf4j.Logger;
42 +
43 +import com.google.common.base.MoreObjects;
44 +import com.google.common.base.Throwables;
45 +import com.google.common.collect.ImmutableList;
46 +import com.google.common.collect.Maps;
47 +import com.google.common.collect.Queues;
48 +import com.google.common.util.concurrent.AtomicLongMap;
49 +
50 +import io.atomix.copycat.server.Commit;
51 +import io.atomix.copycat.server.Snapshottable;
52 +import io.atomix.copycat.server.StateMachineExecutor;
53 +import io.atomix.copycat.server.session.ServerSession;
54 +import io.atomix.copycat.server.session.SessionListener;
55 +import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
56 +import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
57 +import io.atomix.resource.ResourceStateMachine;
58 +
59 +/**
60 + * State machine for {@link AtomixWorkQueue} resource.
61 + */
62 +public class AtomixWorkQueueState extends ResourceStateMachine implements SessionListener, Snapshottable {
63 +
64 + private final Logger log = getLogger(getClass());
65 +
66 + private final AtomicLong totalCompleted = new AtomicLong(0);
67 +
68 + private final Queue<TaskHolder> unassignedTasks = Queues.newArrayDeque();
69 + private final Map<String, TaskAssignment> assignments = Maps.newHashMap();
70 + private final Map<Long, Commit<? extends Register>> registeredWorkers = Maps.newHashMap();
71 + private final AtomicLongMap<Long> activeTasksPerSession = AtomicLongMap.create();
72 +
73 + protected AtomixWorkQueueState(Properties config) {
74 + super(config);
75 + }
76 +
77 + @Override
78 + protected void configure(StateMachineExecutor executor) {
79 + executor.register(Stats.class, this::stats);
80 + executor.register(Register.class, (Consumer<Commit<Register>>) this::register);
81 + executor.register(Unregister.class, (Consumer<Commit<Unregister>>) this::unregister);
82 + executor.register(Add.class, (Consumer<Commit<Add>>) this::add);
83 + executor.register(Take.class, this::take);
84 + executor.register(Complete.class, (Consumer<Commit<Complete>>) this::complete);
85 + }
86 +
87 + protected WorkQueueStats stats(Commit<? extends Stats> commit) {
88 + try {
89 + return WorkQueueStats.builder()
90 + .withTotalCompleted(totalCompleted.get())
91 + .withTotalPending(unassignedTasks.size())
92 + .withTotalInProgress(assignments.size())
93 + .build();
94 + } finally {
95 + commit.close();
96 + }
97 + }
98 +
99 + protected void register(Commit<? extends Register> commit) {
100 + long sessionId = commit.session().id();
101 + if (registeredWorkers.putIfAbsent(sessionId, commit) != null) {
102 + commit.close();
103 + }
104 + }
105 +
106 + protected void unregister(Commit<? extends Unregister> commit) {
107 + try {
108 + Commit<? extends Register> registerCommit = registeredWorkers.remove(commit.session().id());
109 + if (registerCommit != null) {
110 + registerCommit.close();
111 + }
112 + } finally {
113 + commit.close();
114 + }
115 + }
116 +
117 + protected void add(Commit<? extends Add> commit) {
118 + Collection<byte[]> items = commit.operation().items();
119 +
120 + // Create a CountDownCompleter that will close the commit when all tasks
121 + // submitted as part of it are completed.
122 + CountDownCompleter<Commit<? extends Add>> referenceTracker =
123 + new CountDownCompleter<>(commit, items.size(), Commit::close);
124 +
125 + AtomicInteger itemIndex = new AtomicInteger(0);
126 + items.forEach(item -> {
127 + String taskId = String.format("%d:%d:%d", commit.session().id(),
128 + commit.index(),
129 + itemIndex.getAndIncrement());
130 + unassignedTasks.add(new TaskHolder(new Task<>(taskId, item), referenceTracker));
131 + });
132 +
133 + // Send an event to all sessions that have expressed interest in task processing
134 + // and are not actively processing a task.
135 + registeredWorkers.values()
136 + .stream()
137 + .map(Commit::session)
138 + .forEach(session -> session.publish(AtomixWorkQueue.TASK_AVAILABLE));
139 + // FIXME: This generates a lot of event traffic.
140 + }
141 +
142 + protected Collection<Task<byte[]>> take(Commit<? extends Take> commit) {
143 + try {
144 + if (unassignedTasks.isEmpty()) {
145 + return ImmutableList.of();
146 + }
147 + long sessionId = commit.session().id();
148 + int maxTasks = commit.operation().maxTasks();
149 + return IntStream.range(0, Math.min(maxTasks, unassignedTasks.size()))
150 + .mapToObj(i -> {
151 + TaskHolder holder = unassignedTasks.poll();
152 + String taskId = holder.task().taskId();
153 + TaskAssignment assignment = new TaskAssignment(sessionId, holder);
154 +
155 + // bookkeeping
156 + assignments.put(taskId, assignment);
157 + activeTasksPerSession.incrementAndGet(sessionId);
158 +
159 + return holder.task();
160 + })
161 + .collect(Collectors.toCollection(ArrayList::new));
162 + } catch (Exception e) {
163 + log.warn("State machine update failed", e);
164 + throw Throwables.propagate(e);
165 + } finally {
166 + commit.close();
167 + }
168 + }
169 +
170 + protected void complete(Commit<? extends Complete> commit) {
171 + long sessionId = commit.session().id();
172 + try {
173 + commit.operation().taskIds().forEach(taskId -> {
174 + TaskAssignment assignment = assignments.get(taskId);
175 + if (assignment != null) {
176 + assignments.remove(taskId).markComplete();
177 + // bookkeeping
178 + totalCompleted.incrementAndGet();
179 + activeTasksPerSession.decrementAndGet(sessionId);
180 + }
181 + });
182 + } catch (Exception e) {
183 + log.warn("State machine update failed", e);
184 + throw Throwables.propagate(e);
185 + } finally {
186 + commit.close();
187 + }
188 + }
189 +
190 + @Override
191 + public void register(ServerSession session) {
192 + }
193 +
194 + @Override
195 + public void unregister(ServerSession session) {
196 + evictWorker(session.id());
197 + }
198 +
199 + @Override
200 + public void expire(ServerSession session) {
201 + evictWorker(session.id());
202 + }
203 +
204 + @Override
205 + public void close(ServerSession session) {
206 + evictWorker(session.id());
207 + }
208 +
209 + @Override
210 + public void snapshot(SnapshotWriter writer) {
211 + writer.writeLong(totalCompleted.get());
212 + }
213 +
214 + @Override
215 + public void install(SnapshotReader reader) {
216 + totalCompleted.set(reader.readLong());
217 + }
218 +
219 + private void evictWorker(long sessionId) {
220 + Commit<? extends Register> commit = registeredWorkers.remove(sessionId);
221 + if (commit != null) {
222 + commit.close();
223 + }
224 +
225 + // TODO: Maintain an index of tasks by session for efficient access.
226 + Iterator<Map.Entry<String, TaskAssignment>> iter = assignments.entrySet().iterator();
227 + while (iter.hasNext()) {
228 + Map.Entry<String, TaskAssignment> entry = iter.next();
229 + TaskAssignment assignment = entry.getValue();
230 + if (assignment.sessionId() == sessionId) {
231 + unassignedTasks.add(assignment.taskHolder());
232 + iter.remove();
233 + }
234 + }
235 +
236 + // Bookkeeping
237 + activeTasksPerSession.remove(sessionId);
238 + activeTasksPerSession.removeAllZeros();
239 + }
240 +
241 + private class TaskHolder {
242 +
243 + private final Task<byte[]> task;
244 + private final CountDownCompleter<Commit<? extends Add>> referenceTracker;
245 +
246 + public TaskHolder(Task<byte[]> delegate, CountDownCompleter<Commit<? extends Add>> referenceTracker) {
247 + this.task = delegate;
248 + this.referenceTracker = referenceTracker;
249 + }
250 +
251 + public Task<byte[]> task() {
252 + return task;
253 + }
254 +
255 + public void complete() {
256 + referenceTracker.countDown();
257 + }
258 + }
259 +
260 + private class TaskAssignment {
261 + private final long sessionId;
262 + private final TaskHolder taskHolder;
263 +
264 + public TaskAssignment(long sessionId, TaskHolder taskHolder) {
265 + this.sessionId = sessionId;
266 + this.taskHolder = taskHolder;
267 + }
268 +
269 + public long sessionId() {
270 + return sessionId;
271 + }
272 +
273 + public TaskHolder taskHolder() {
274 + return taskHolder;
275 + }
276 +
277 + public void markComplete() {
278 + taskHolder.complete();
279 + }
280 +
281 + @Override
282 + public String toString() {
283 + return MoreObjects.toStringHelper(getClass())
284 + .add("sessionId", sessionId)
285 + .add("taskHolder", taskHolder)
286 + .toString();
287 + }
288 + }
289 +}
...@@ -16,6 +16,7 @@ ...@@ -16,6 +16,7 @@
16 package org.onosproject.store.primitives.resources.impl; 16 package org.onosproject.store.primitives.resources.impl;
17 17
18 import com.google.common.util.concurrent.Uninterruptibles; 18 import com.google.common.util.concurrent.Uninterruptibles;
19 +
19 import io.atomix.AtomixClient; 20 import io.atomix.AtomixClient;
20 import io.atomix.catalyst.serializer.Serializer; 21 import io.atomix.catalyst.serializer.Serializer;
21 import io.atomix.catalyst.transport.Address; 22 import io.atomix.catalyst.transport.Address;
...@@ -116,16 +117,16 @@ public abstract class AtomixTestBase { ...@@ -116,16 +117,16 @@ public abstract class AtomixTestBase {
116 CompletableFuture.allOf(atomixClients.stream() 117 CompletableFuture.allOf(atomixClients.stream()
117 .map(AtomixClient::close) 118 .map(AtomixClient::close)
118 .toArray(CompletableFuture[]::new)); 119 .toArray(CompletableFuture[]::new));
120 + closeClients.join();
119 121
120 - closeClients 122 + CompletableFuture<Void> closeServers =
121 - .thenCompose(v -> CompletableFuture 123 + CompletableFuture.allOf(copycatServers.stream()
122 - .allOf(copycatServers.stream()
123 .map(CopycatServer::shutdown) 124 .map(CopycatServer::shutdown)
124 - .toArray(CompletableFuture[]::new))).join(); 125 + .toArray(CompletableFuture[]::new));
125 - 126 + closeServers.join();
126 - atomixClients = new ArrayList<>();
127 127
128 - copycatServers = new ArrayList<>(); 128 + atomixClients.clear();
129 + copycatServers.clear();
129 } 130 }
130 131
131 132
......
1 +/*
2 + * Copyright 2016-present 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.time.Duration;
19 +import java.util.Arrays;
20 +import java.util.UUID;
21 +import java.util.concurrent.CountDownLatch;
22 +import java.util.concurrent.Executor;
23 +import java.util.concurrent.Executors;
24 +import java.util.concurrent.TimeUnit;
25 +
26 +import io.atomix.Atomix;
27 +import io.atomix.AtomixClient;
28 +import io.atomix.resource.ResourceType;
29 +
30 +import org.junit.AfterClass;
31 +import org.junit.BeforeClass;
32 +import org.junit.Test;
33 +import org.onlab.util.Tools;
34 +import org.onosproject.store.service.Task;
35 +import org.onosproject.store.service.WorkQueueStats;
36 +
37 +import com.google.common.util.concurrent.Uninterruptibles;
38 +
39 +import static org.junit.Assert.assertEquals;
40 +import static org.junit.Assert.assertNull;
41 +import static org.junit.Assert.assertTrue;
42 +
43 +/**
44 + * Unit tests for {@link AtomixWorkQueue}.
45 + */
46 +public class AtomixWorkQueueTest extends AtomixTestBase {
47 +
48 + private static final Duration DEFAULT_PROCESSING_TIME = Duration.ofMillis(100);
49 + private static final byte[] DEFAULT_PAYLOAD = "hello world".getBytes();
50 +
51 + @BeforeClass
52 + public static void preTestSetup() throws Throwable {
53 + createCopycatServers(1);
54 + }
55 +
56 + @AfterClass
57 + public static void postTestCleanup() throws Exception {
58 + clearTests();
59 + }
60 +
61 + @Override
62 + protected ResourceType resourceType() {
63 + return new ResourceType(AtomixWorkQueue.class);
64 + }
65 +
66 + @Test
67 + public void testAdd() throws Throwable {
68 + String queueName = UUID.randomUUID().toString();
69 + Atomix atomix1 = createAtomixClient();
70 + AtomixWorkQueue queue1 = atomix1.getResource(queueName, AtomixWorkQueue.class).join();
71 + byte[] item = DEFAULT_PAYLOAD;
72 + queue1.addOne(item).join();
73 +
74 + Atomix atomix2 = createAtomixClient();
75 + AtomixWorkQueue queue2 = atomix2.getResource(queueName, AtomixWorkQueue.class).join();
76 + byte[] task2 = DEFAULT_PAYLOAD;
77 + queue2.addOne(task2).join();
78 +
79 + WorkQueueStats stats = queue1.stats().join();
80 + assertEquals(stats.totalPending(), 2);
81 + assertEquals(stats.totalInProgress(), 0);
82 + assertEquals(stats.totalCompleted(), 0);
83 + }
84 +
85 + @Test
86 + public void testAddMultiple() throws Throwable {
87 + String queueName = UUID.randomUUID().toString();
88 + Atomix atomix1 = createAtomixClient();
89 + AtomixWorkQueue queue1 = atomix1.getResource(queueName, AtomixWorkQueue.class).join();
90 + byte[] item1 = DEFAULT_PAYLOAD;
91 + byte[] item2 = DEFAULT_PAYLOAD;
92 + queue1.addMultiple(Arrays.asList(item1, item2)).join();
93 +
94 + WorkQueueStats stats = queue1.stats().join();
95 + assertEquals(stats.totalPending(), 2);
96 + assertEquals(stats.totalInProgress(), 0);
97 + assertEquals(stats.totalCompleted(), 0);
98 + }
99 +
100 + @Test
101 + public void testTakeAndComplete() throws Throwable {
102 + String queueName = UUID.randomUUID().toString();
103 + Atomix atomix1 = createAtomixClient();
104 + AtomixWorkQueue queue1 = atomix1.getResource(queueName, AtomixWorkQueue.class).join();
105 + byte[] item1 = DEFAULT_PAYLOAD;
106 + queue1.addOne(item1).join();
107 +
108 + Atomix atomix2 = createAtomixClient();
109 + AtomixWorkQueue queue2 = atomix2.getResource(queueName, AtomixWorkQueue.class).join();
110 + Task<byte[]> removedTask = queue2.take().join();
111 +
112 + WorkQueueStats stats = queue2.stats().join();
113 + assertEquals(stats.totalPending(), 0);
114 + assertEquals(stats.totalInProgress(), 1);
115 + assertEquals(stats.totalCompleted(), 0);
116 +
117 + assertTrue(Arrays.equals(removedTask.payload(), item1));
118 + queue2.complete(Arrays.asList(removedTask.taskId())).join();
119 +
120 + stats = queue1.stats().join();
121 + assertEquals(stats.totalPending(), 0);
122 + assertEquals(stats.totalInProgress(), 0);
123 + assertEquals(stats.totalCompleted(), 1);
124 +
125 + // Another take should return null
126 + assertNull(queue2.take().join());
127 + }
128 +
129 + @Test
130 + public void testUnexpectedClientClose() throws Throwable {
131 + String queueName = UUID.randomUUID().toString();
132 + Atomix atomix1 = createAtomixClient();
133 + AtomixWorkQueue queue1 = atomix1.getResource(queueName, AtomixWorkQueue.class).join();
134 + byte[] item1 = DEFAULT_PAYLOAD;
135 + queue1.addOne(item1).join();
136 +
137 + AtomixClient atomix2 = createAtomixClient();
138 + AtomixWorkQueue queue2 = atomix2.getResource(queueName, AtomixWorkQueue.class).join();
139 + queue2.take().join();
140 +
141 + WorkQueueStats stats = queue1.stats().join();
142 + assertEquals(0, stats.totalPending());
143 + assertEquals(1, stats.totalInProgress());
144 + assertEquals(0, stats.totalCompleted());
145 +
146 + atomix2.close().join();
147 +
148 + stats = queue1.stats().join();
149 + assertEquals(1, stats.totalPending());
150 + assertEquals(0, stats.totalInProgress());
151 + assertEquals(0, stats.totalCompleted());
152 + }
153 +
154 + @Test
155 + public void testAutomaticTaskProcessing() throws Throwable {
156 + String queueName = UUID.randomUUID().toString();
157 + Atomix atomix1 = createAtomixClient();
158 + AtomixWorkQueue queue1 = atomix1.getResource(queueName, AtomixWorkQueue.class).join();
159 + Executor executor = Executors.newSingleThreadExecutor();
160 +
161 + CountDownLatch latch1 = new CountDownLatch(1);
162 + queue1.registerTaskProcessor(s -> latch1.countDown(), 2, executor);
163 +
164 + AtomixClient atomix2 = createAtomixClient();
165 + AtomixWorkQueue queue2 = atomix2.getResource(queueName, AtomixWorkQueue.class).join();
166 + byte[] item1 = DEFAULT_PAYLOAD;
167 + queue2.addOne(item1).join();
168 +
169 + Uninterruptibles.awaitUninterruptibly(latch1, 500, TimeUnit.MILLISECONDS);
170 + queue1.stopProcessing();
171 +
172 + byte[] item2 = DEFAULT_PAYLOAD;
173 + byte[] item3 = DEFAULT_PAYLOAD;
174 +
175 + Tools.delay((int) DEFAULT_PROCESSING_TIME.toMillis());
176 +
177 + queue2.addMultiple(Arrays.asList(item2, item3)).join();
178 +
179 + WorkQueueStats stats = queue1.stats().join();
180 + assertEquals(2, stats.totalPending());
181 + assertEquals(0, stats.totalInProgress());
182 + assertEquals(1, stats.totalCompleted());
183 +
184 + CountDownLatch latch2 = new CountDownLatch(2);
185 + queue1.registerTaskProcessor(s -> latch2.countDown(), 2, executor);
186 +
187 + Uninterruptibles.awaitUninterruptibly(latch2, 500, TimeUnit.MILLISECONDS);
188 + }
189 +}
...@@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableList; ...@@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableList;
19 import com.google.common.collect.ImmutableMap; 19 import com.google.common.collect.ImmutableMap;
20 import com.google.common.collect.ImmutableSet; 20 import com.google.common.collect.ImmutableSet;
21 import com.google.common.collect.Maps; 21 import com.google.common.collect.Maps;
22 +
22 import org.onlab.packet.ChassisId; 23 import org.onlab.packet.ChassisId;
23 import org.onlab.packet.EthType; 24 import org.onlab.packet.EthType;
24 import org.onlab.packet.Ip4Address; 25 import org.onlab.packet.Ip4Address;
...@@ -208,7 +209,9 @@ import org.onosproject.store.primitives.TransactionId; ...@@ -208,7 +209,9 @@ import org.onosproject.store.primitives.TransactionId;
208 import org.onosproject.store.service.MapEvent; 209 import org.onosproject.store.service.MapEvent;
209 import org.onosproject.store.service.MapTransaction; 210 import org.onosproject.store.service.MapTransaction;
210 import org.onosproject.store.service.SetEvent; 211 import org.onosproject.store.service.SetEvent;
212 +import org.onosproject.store.service.Task;
211 import org.onosproject.store.service.Versioned; 213 import org.onosproject.store.service.Versioned;
214 +import org.onosproject.store.service.WorkQueueStats;
212 215
213 import java.net.URI; 216 import java.net.URI;
214 import java.time.Duration; 217 import java.time.Duration;
...@@ -338,6 +341,8 @@ public final class KryoNamespaces { ...@@ -338,6 +341,8 @@ public final class KryoNamespaces {
338 Leadership.class, 341 Leadership.class,
339 LeadershipEvent.class, 342 LeadershipEvent.class,
340 LeadershipEvent.Type.class, 343 LeadershipEvent.Type.class,
344 + Task.class,
345 + WorkQueueStats.class,
341 HostId.class, 346 HostId.class,
342 HostDescription.class, 347 HostDescription.class,
343 DefaultHostDescription.class, 348 DefaultHostDescription.class,
......