Madan Jampani

Copycat messaging new happens over the same cluster messaging used for all other…

… ONOS p2p communication
...@@ -26,7 +26,7 @@ public interface DatabaseService { ...@@ -26,7 +26,7 @@ public interface DatabaseService {
26 26
27 /** 27 /**
28 * Performs a write operation on the database. 28 * Performs a write operation on the database.
29 - * @param request 29 + * @param request write request
30 * @return write result. 30 * @return write result.
31 * @throws DatabaseException if there is failure in execution write. 31 * @throws DatabaseException if there is failure in execution write.
32 */ 32 */
......
...@@ -17,15 +17,15 @@ public class ReadResult { ...@@ -17,15 +17,15 @@ public class ReadResult {
17 } 17 }
18 18
19 /** 19 /**
20 - * Database table name. 20 + * Returns database table name.
21 - * @return 21 + * @return table name.
22 */ 22 */
23 public String tableName() { 23 public String tableName() {
24 return tableName; 24 return tableName;
25 } 25 }
26 26
27 /** 27 /**
28 - * Database table key. 28 + * Returns database table key.
29 * @return key. 29 * @return key.
30 */ 30 */
31 public String key() { 31 public String key() {
...@@ -33,7 +33,7 @@ public class ReadResult { ...@@ -33,7 +33,7 @@ public class ReadResult {
33 } 33 }
34 34
35 /** 35 /**
36 - * value associated with the key. 36 + * Returns value associated with the key.
37 * @return non-null value if the table contains one, null otherwise. 37 * @return non-null value if the table contains one, null otherwise.
38 */ 38 */
39 public VersionedValue value() { 39 public VersionedValue value() {
......
...@@ -12,8 +12,8 @@ public class VersionedValue { ...@@ -12,8 +12,8 @@ public class VersionedValue {
12 12
13 /** 13 /**
14 * Creates a new instance with the specified value and version. 14 * Creates a new instance with the specified value and version.
15 - * @param value 15 + * @param value value
16 - * @param version 16 + * @param version version
17 */ 17 */
18 public VersionedValue(byte[] value, long version) { 18 public VersionedValue(byte[] value, long version) {
19 this.value = value; 19 this.value = value;
......
1 package org.onlab.onos.store.service.impl; 1 package org.onlab.onos.store.service.impl;
2 2
3 +import static com.google.common.base.Preconditions.checkNotNull;
4 +import static org.slf4j.LoggerFactory.getLogger;
5 +
3 import java.util.ArrayList; 6 import java.util.ArrayList;
4 import java.util.Arrays; 7 import java.util.Arrays;
5 import java.util.Collection; 8 import java.util.Collection;
...@@ -27,6 +30,16 @@ import net.kuujo.copycat.spi.protocol.Protocol; ...@@ -27,6 +30,16 @@ import net.kuujo.copycat.spi.protocol.Protocol;
27 import net.kuujo.copycat.spi.protocol.ProtocolClient; 30 import net.kuujo.copycat.spi.protocol.ProtocolClient;
28 import net.kuujo.copycat.spi.protocol.ProtocolServer; 31 import net.kuujo.copycat.spi.protocol.ProtocolServer;
29 32
33 +import org.apache.felix.scr.annotations.Activate;
34 +import org.apache.felix.scr.annotations.Component;
35 +import org.apache.felix.scr.annotations.Deactivate;
36 +import org.apache.felix.scr.annotations.Reference;
37 +import org.apache.felix.scr.annotations.ReferenceCardinality;
38 +import org.apache.felix.scr.annotations.Service;
39 +import org.onlab.onos.cluster.ClusterService;
40 +import org.onlab.onos.cluster.ControllerNode;
41 +import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
42 +import org.onlab.onos.store.cluster.messaging.MessageSubject;
30 import org.onlab.onos.store.serializers.ImmutableListSerializer; 43 import org.onlab.onos.store.serializers.ImmutableListSerializer;
31 import org.onlab.onos.store.serializers.ImmutableMapSerializer; 44 import org.onlab.onos.store.serializers.ImmutableMapSerializer;
32 import org.onlab.onos.store.serializers.ImmutableSetSerializer; 45 import org.onlab.onos.store.serializers.ImmutableSetSerializer;
...@@ -37,6 +50,7 @@ import org.onlab.onos.store.service.VersionedValue; ...@@ -37,6 +50,7 @@ import org.onlab.onos.store.service.VersionedValue;
37 import org.onlab.onos.store.service.WriteRequest; 50 import org.onlab.onos.store.service.WriteRequest;
38 import org.onlab.onos.store.service.WriteResult; 51 import org.onlab.onos.store.service.WriteResult;
39 import org.onlab.util.KryoNamespace; 52 import org.onlab.util.KryoNamespace;
53 +import org.slf4j.Logger;
40 54
41 import com.esotericsoftware.kryo.Kryo; 55 import com.esotericsoftware.kryo.Kryo;
42 import com.esotericsoftware.kryo.io.Input; 56 import com.esotericsoftware.kryo.io.Input;
...@@ -46,17 +60,44 @@ import com.google.common.collect.ImmutableMap; ...@@ -46,17 +60,44 @@ import com.google.common.collect.ImmutableMap;
46 import com.google.common.collect.ImmutableSet; 60 import com.google.common.collect.ImmutableSet;
47 61
48 /** 62 /**
49 - * {@link Protocol} based on {@link org.onlab.netty.NettyMessagingService}. 63 + * Licensed to the Apache Software Foundation (ASF) under one
64 + * or more contributor license agreements. See the NOTICE file
65 + * distributed with this work for additional information
66 + * regarding copyright ownership. The ASF licenses this file
67 + * to you under the Apache License, Version 2.0 (the
68 + * "License"); you may not use this file except in compliance
69 + * with the License. You may obtain a copy of the License at
70 + *
71 + * http://www.apache.org/licenses/LICENSE-2.0
72 +
73 + * Unless required by applicable law or agreed to in writing,
74 + * software distributed under the License is distributed on an
75 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
76 + * KIND, either express or implied. See the License for the
77 + * specific language governing permissions and limitations under
78 + * the License.
50 */ 79 */
51 -public class NettyProtocol implements Protocol<TcpMember> {
52 80
53 - public static final String COPYCAT_PING = "copycat-raft-consensus-ping"; 81 +@Component(immediate = true)
54 - public static final String COPYCAT_SYNC = "copycat-raft-consensus-sync"; 82 +@Service
55 - public static final String COPYCAT_POLL = "copycat-raft-consensus-poll"; 83 +public class ClusterMessagingProtocol implements Protocol<TcpMember> {
56 - public static final String COPYCAT_SUBMIT = "copycat-raft-consensus-submit"; 84 +
85 + private final Logger log = getLogger(getClass());
86 +
87 + @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
88 + ClusterService clusterService;
89 +
90 + @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
91 + ClusterCommunicationService clusterCommunicator;
57 92
58 - // TODO: make this configurable. 93 + public static final MessageSubject COPYCAT_PING =
59 - public static final long RETRY_INTERVAL_MILLIS = 2000; 94 + new MessageSubject("copycat-raft-consensus-ping");
95 + public static final MessageSubject COPYCAT_SYNC =
96 + new MessageSubject("copycat-raft-consensus-sync");
97 + public static final MessageSubject COPYCAT_POLL =
98 + new MessageSubject("copycat-raft-consensus-poll");
99 + public static final MessageSubject COPYCAT_SUBMIT =
100 + new MessageSubject("copycat-raft-consensus-submit");
60 101
61 private static final KryoNamespace COPYCAT = KryoNamespace.newBuilder() 102 private static final KryoNamespace COPYCAT = KryoNamespace.newBuilder()
62 .register(PingRequest.class) 103 .register(PingRequest.class)
...@@ -76,8 +117,7 @@ public class NettyProtocol implements Protocol<TcpMember> { ...@@ -76,8 +117,7 @@ public class NettyProtocol implements Protocol<TcpMember> {
76 .register(TcpMember.class) 117 .register(TcpMember.class)
77 .build(); 118 .build();
78 119
79 - // TODO: Move to the right place. 120 + private static final KryoNamespace DATABASE = KryoNamespace.newBuilder()
80 - private static final KryoNamespace CRAFT = KryoNamespace.newBuilder()
81 .register(ReadRequest.class) 121 .register(ReadRequest.class)
82 .register(WriteRequest.class) 122 .register(WriteRequest.class)
83 .register(InternalReadResult.class) 123 .register(InternalReadResult.class)
...@@ -116,31 +156,41 @@ public class NettyProtocol implements Protocol<TcpMember> { ...@@ -116,31 +156,41 @@ public class NettyProtocol implements Protocol<TcpMember> {
116 serializerPool = KryoNamespace.newBuilder() 156 serializerPool = KryoNamespace.newBuilder()
117 .register(COPYCAT) 157 .register(COPYCAT)
118 .register(COMMON) 158 .register(COMMON)
119 - .register(CRAFT) 159 + .register(DATABASE)
120 .build() 160 .build()
121 .populate(1); 161 .populate(1);
122 } 162 }
123 }; 163 };
124 164
125 - private NettyProtocolServer server = null; 165 + @Activate
166 + public void activate() {
167 + log.info("Started.");
168 + }
126 169
127 - // FIXME: This is a total hack.Assumes 170 + @Deactivate
128 - // ProtocolServer is initialized before ProtocolClient 171 + public void deactivate() {
129 - protected NettyProtocolServer getServer() { 172 + log.info("Stopped.");
130 - if (server == null) {
131 - throw new IllegalStateException("ProtocolServer is not initialized yet!");
132 - }
133 - return server;
134 } 173 }
135 174
136 @Override 175 @Override
137 public ProtocolServer createServer(TcpMember member) { 176 public ProtocolServer createServer(TcpMember member) {
138 - server = new NettyProtocolServer(member); 177 + return new ClusterMessagingProtocolServer(clusterCommunicator);
139 - return server;
140 } 178 }
141 179
142 @Override 180 @Override
143 public ProtocolClient createClient(TcpMember member) { 181 public ProtocolClient createClient(TcpMember member) {
144 - return new NettyProtocolClient(this, member); 182 + ControllerNode node = getControllerNode(member.host(), member.port());
183 + checkNotNull(node, "A valid controller node is expected");
184 + return new ClusterMessagingProtocolClient(
185 + clusterCommunicator, node);
186 + }
187 +
188 + private ControllerNode getControllerNode(String host, int port) {
189 + for (ControllerNode node : clusterService.getNodes()) {
190 + if (node.ip().toString().equals(host) && node.tcpPort() == port) {
191 + return node;
192 + }
193 + }
194 + return null;
145 } 195 }
146 -} 196 +}
...\ No newline at end of file ...\ No newline at end of file
......
...@@ -11,7 +11,6 @@ import java.util.concurrent.ThreadFactory; ...@@ -11,7 +11,6 @@ import java.util.concurrent.ThreadFactory;
11 import java.util.concurrent.TimeUnit; 11 import java.util.concurrent.TimeUnit;
12 import java.util.concurrent.TimeoutException; 12 import java.util.concurrent.TimeoutException;
13 13
14 -import net.kuujo.copycat.cluster.TcpMember;
15 import net.kuujo.copycat.protocol.PingRequest; 14 import net.kuujo.copycat.protocol.PingRequest;
16 import net.kuujo.copycat.protocol.PingResponse; 15 import net.kuujo.copycat.protocol.PingResponse;
17 import net.kuujo.copycat.protocol.PollRequest; 16 import net.kuujo.copycat.protocol.PollRequest;
...@@ -22,37 +21,54 @@ import net.kuujo.copycat.protocol.SyncRequest; ...@@ -22,37 +21,54 @@ import net.kuujo.copycat.protocol.SyncRequest;
22 import net.kuujo.copycat.protocol.SyncResponse; 21 import net.kuujo.copycat.protocol.SyncResponse;
23 import net.kuujo.copycat.spi.protocol.ProtocolClient; 22 import net.kuujo.copycat.spi.protocol.ProtocolClient;
24 23
25 -import org.onlab.netty.Endpoint; 24 +import org.onlab.onos.cluster.ControllerNode;
26 -import org.onlab.netty.NettyMessagingService; 25 +import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
26 +import org.onlab.onos.store.cluster.messaging.ClusterMessage;
27 +import org.onlab.onos.store.cluster.messaging.MessageSubject;
27 import org.slf4j.Logger; 28 import org.slf4j.Logger;
28 29
29 import com.google.common.util.concurrent.ThreadFactoryBuilder; 30 import com.google.common.util.concurrent.ThreadFactoryBuilder;
30 31
31 /** 32 /**
32 - * {@link NettyMessagingService} based Copycat protocol client. 33 + * Licensed to the Apache Software Foundation (ASF) under one
34 + * or more contributor license agreements. See the NOTICE file
35 + * distributed with this work for additional information
36 + * regarding copyright ownership. The ASF licenses this file
37 + * to you under the Apache License, Version 2.0 (the
38 + * "License"); you may not use this file except in compliance
39 + * with the License. You may obtain a copy of the License at
40 + *
41 + * http://www.apache.org/licenses/LICENSE-2.0
42 +
43 + * Unless required by applicable law or agreed to in writing,
44 + * software distributed under the License is distributed on an
45 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
46 + * KIND, either express or implied. See the License for the
47 + * specific language governing permissions and limitations under
48 + * the License.
33 */ 49 */
34 -public class NettyProtocolClient implements ProtocolClient { 50 +
51 +public class ClusterMessagingProtocolClient implements ProtocolClient {
35 52
36 private final Logger log = getLogger(getClass()); 53 private final Logger log = getLogger(getClass());
54 +
37 private static final ThreadFactory THREAD_FACTORY = 55 private static final ThreadFactory THREAD_FACTORY =
38 new ThreadFactoryBuilder().setNameFormat("copycat-netty-messaging-%d").build(); 56 new ThreadFactoryBuilder().setNameFormat("copycat-netty-messaging-%d").build();
39 57
40 - // Remote endpoint, this client instance is used 58 + public static final long RETRY_INTERVAL_MILLIS = 2000;
41 - // for communicating with. 59 +
42 - private final Endpoint remoteEp; 60 + private final ClusterCommunicationService clusterCommunicator;
43 - private final NettyMessagingService messagingService; 61 + private final ControllerNode remoteNode;
44 62
45 - // TODO: Is 10 the right number of threads? 63 + // FIXME: Thread pool sizing.
46 private static final ScheduledExecutorService THREAD_POOL = 64 private static final ScheduledExecutorService THREAD_POOL =
47 new ScheduledThreadPoolExecutor(10, THREAD_FACTORY); 65 new ScheduledThreadPoolExecutor(10, THREAD_FACTORY);
48 66
49 - public NettyProtocolClient(NettyProtocol protocol, TcpMember member) { 67 + public ClusterMessagingProtocolClient(
50 - this(new Endpoint(member.host(), member.port()), protocol.getServer().getNettyMessagingService()); 68 + ClusterCommunicationService clusterCommunicator,
51 - } 69 + ControllerNode remoteNode) {
52 - 70 + this.clusterCommunicator = clusterCommunicator;
53 - public NettyProtocolClient(Endpoint remoteEp, NettyMessagingService messagingService) { 71 + this.remoteNode = remoteNode;
54 - this.remoteEp = remoteEp;
55 - this.messagingService = messagingService;
56 } 72 }
57 73
58 @Override 74 @Override
...@@ -85,16 +101,16 @@ public class NettyProtocolClient implements ProtocolClient { ...@@ -85,16 +101,16 @@ public class NettyProtocolClient implements ProtocolClient {
85 return CompletableFuture.completedFuture(null); 101 return CompletableFuture.completedFuture(null);
86 } 102 }
87 103
88 - public <I> String messageType(I input) { 104 + public <I> MessageSubject messageType(I input) {
89 Class<?> clazz = input.getClass(); 105 Class<?> clazz = input.getClass();
90 if (clazz.equals(PollRequest.class)) { 106 if (clazz.equals(PollRequest.class)) {
91 - return NettyProtocol.COPYCAT_POLL; 107 + return ClusterMessagingProtocol.COPYCAT_POLL;
92 } else if (clazz.equals(SyncRequest.class)) { 108 } else if (clazz.equals(SyncRequest.class)) {
93 - return NettyProtocol.COPYCAT_SYNC; 109 + return ClusterMessagingProtocol.COPYCAT_SYNC;
94 } else if (clazz.equals(SubmitRequest.class)) { 110 } else if (clazz.equals(SubmitRequest.class)) {
95 - return NettyProtocol.COPYCAT_SUBMIT; 111 + return ClusterMessagingProtocol.COPYCAT_SUBMIT;
96 } else if (clazz.equals(PingRequest.class)) { 112 } else if (clazz.equals(PingRequest.class)) {
97 - return NettyProtocol.COPYCAT_PING; 113 + return ClusterMessagingProtocol.COPYCAT_PING;
98 } else { 114 } else {
99 throw new IllegalArgumentException("Unknown class " + clazz.getName()); 115 throw new IllegalArgumentException("Unknown class " + clazz.getName());
100 } 116 }
...@@ -109,33 +125,34 @@ public class NettyProtocolClient implements ProtocolClient { ...@@ -109,33 +125,34 @@ public class NettyProtocolClient implements ProtocolClient {
109 125
110 private class RPCTask<I, O> implements Runnable { 126 private class RPCTask<I, O> implements Runnable {
111 127
112 - private final String messageType; 128 + private final ClusterMessage message;
113 - private final byte[] payload;
114 -
115 private final CompletableFuture<O> future; 129 private final CompletableFuture<O> future;
116 130
117 public RPCTask(I request, CompletableFuture<O> future) { 131 public RPCTask(I request, CompletableFuture<O> future) {
118 - this.messageType = messageType(request); 132 + this.message =
119 - this.payload = NettyProtocol.SERIALIZER.encode(request); 133 + new ClusterMessage(
134 + null,
135 + messageType(request),
136 + ClusterMessagingProtocol.SERIALIZER.encode(request));
120 this.future = future; 137 this.future = future;
121 } 138 }
122 139
123 @Override 140 @Override
124 public void run() { 141 public void run() {
125 try { 142 try {
126 - byte[] response = messagingService 143 + byte[] response = clusterCommunicator
127 - .sendAndReceive(remoteEp, messageType, payload) 144 + .sendAndReceive(message, remoteNode.id())
128 - .get(NettyProtocol.RETRY_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); 145 + .get(RETRY_INTERVAL_MILLIS, TimeUnit.MILLISECONDS);
129 - future.complete(NettyProtocol.SERIALIZER.decode(response)); 146 + future.complete(ClusterMessagingProtocol.SERIALIZER.decode(response));
130 147
131 } catch (IOException | InterruptedException | ExecutionException | TimeoutException e) { 148 } catch (IOException | InterruptedException | ExecutionException | TimeoutException e) {
132 - if (messageType.equals(NettyProtocol.COPYCAT_SYNC) || 149 + if (message.subject().equals(ClusterMessagingProtocol.COPYCAT_SYNC) ||
133 - messageType.equals(NettyProtocol.COPYCAT_PING)) { 150 + message.subject().equals(ClusterMessagingProtocol.COPYCAT_PING)) {
134 log.warn("Request to {} failed. Will retry " 151 log.warn("Request to {} failed. Will retry "
135 - + "in {} ms", remoteEp, NettyProtocol.RETRY_INTERVAL_MILLIS); 152 + + "in {} ms", remoteNode, RETRY_INTERVAL_MILLIS);
136 THREAD_POOL.schedule( 153 THREAD_POOL.schedule(
137 this, 154 this,
138 - NettyProtocol.RETRY_INTERVAL_MILLIS, 155 + RETRY_INTERVAL_MILLIS,
139 TimeUnit.MILLISECONDS); 156 TimeUnit.MILLISECONDS);
140 } else { 157 } else {
141 future.completeExceptionally(e); 158 future.completeExceptionally(e);
...@@ -145,4 +162,4 @@ public class NettyProtocolClient implements ProtocolClient { ...@@ -145,4 +162,4 @@ public class NettyProtocolClient implements ProtocolClient {
145 } 162 }
146 } 163 }
147 } 164 }
148 -} 165 +}
...\ No newline at end of file ...\ No newline at end of file
......
...@@ -2,10 +2,8 @@ package org.onlab.onos.store.service.impl; ...@@ -2,10 +2,8 @@ package org.onlab.onos.store.service.impl;
2 2
3 import static org.slf4j.LoggerFactory.getLogger; 3 import static org.slf4j.LoggerFactory.getLogger;
4 4
5 -import java.io.IOException;
6 import java.util.concurrent.CompletableFuture; 5 import java.util.concurrent.CompletableFuture;
7 6
8 -import net.kuujo.copycat.cluster.TcpMember;
9 import net.kuujo.copycat.protocol.PingRequest; 7 import net.kuujo.copycat.protocol.PingRequest;
10 import net.kuujo.copycat.protocol.PollRequest; 8 import net.kuujo.copycat.protocol.PollRequest;
11 import net.kuujo.copycat.protocol.RequestHandler; 9 import net.kuujo.copycat.protocol.RequestHandler;
...@@ -13,33 +11,45 @@ import net.kuujo.copycat.protocol.SubmitRequest; ...@@ -13,33 +11,45 @@ import net.kuujo.copycat.protocol.SubmitRequest;
13 import net.kuujo.copycat.protocol.SyncRequest; 11 import net.kuujo.copycat.protocol.SyncRequest;
14 import net.kuujo.copycat.spi.protocol.ProtocolServer; 12 import net.kuujo.copycat.spi.protocol.ProtocolServer;
15 13
16 -import org.onlab.netty.Message; 14 +import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
17 -import org.onlab.netty.MessageHandler; 15 +import org.onlab.onos.store.cluster.messaging.ClusterMessage;
18 -import org.onlab.netty.NettyMessagingService; 16 +import org.onlab.onos.store.cluster.messaging.ClusterMessageHandler;
19 import org.slf4j.Logger; 17 import org.slf4j.Logger;
20 18
21 /** 19 /**
22 - * {@link NettyMessagingService} based Copycat protocol server. 20 + * Licensed to the Apache Software Foundation (ASF) under one
21 + * or more contributor license agreements. See the NOTICE file
22 + * distributed with this work for additional information
23 + * regarding copyright ownership. The ASF licenses this file
24 + * to you under the Apache License, Version 2.0 (the
25 + * "License"); you may not use this file except in compliance
26 + * with the License. You may obtain a copy of the License at
27 + *
28 + * http://www.apache.org/licenses/LICENSE-2.0
29 +
30 + * Unless required by applicable law or agreed to in writing,
31 + * software distributed under the License is distributed on an
32 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
33 + * KIND, either express or implied. See the License for the
34 + * specific language governing permissions and limitations under
35 + * the License.
23 */ 36 */
24 -public class NettyProtocolServer implements ProtocolServer {
25 37
26 - private final Logger log = getLogger(getClass()); 38 +public class ClusterMessagingProtocolServer implements ProtocolServer {
27 39
28 - private final NettyMessagingService messagingService; 40 + private final Logger log = getLogger(getClass());
29 private RequestHandler handler; 41 private RequestHandler handler;
30 42
43 + public ClusterMessagingProtocolServer(ClusterCommunicationService clusterCommunicator) {
31 44
32 - public NettyProtocolServer(TcpMember member) { 45 + clusterCommunicator.addSubscriber(
33 - messagingService = new NettyMessagingService(member.host(), member.port()); 46 + ClusterMessagingProtocol.COPYCAT_PING, new CopycatMessageHandler<PingRequest>());
34 - 47 + clusterCommunicator.addSubscriber(
35 - messagingService.registerHandler(NettyProtocol.COPYCAT_PING, new CopycatMessageHandler<PingRequest>()); 48 + ClusterMessagingProtocol.COPYCAT_SYNC, new CopycatMessageHandler<SyncRequest>());
36 - messagingService.registerHandler(NettyProtocol.COPYCAT_SYNC, new CopycatMessageHandler<SyncRequest>()); 49 + clusterCommunicator.addSubscriber(
37 - messagingService.registerHandler(NettyProtocol.COPYCAT_POLL, new CopycatMessageHandler<PollRequest>()); 50 + ClusterMessagingProtocol.COPYCAT_POLL, new CopycatMessageHandler<PollRequest>());
38 - messagingService.registerHandler(NettyProtocol.COPYCAT_SUBMIT, new CopycatMessageHandler<SubmitRequest>()); 51 + clusterCommunicator.addSubscriber(
39 - } 52 + ClusterMessagingProtocol.COPYCAT_SUBMIT, new CopycatMessageHandler<SubmitRequest>());
40 -
41 - protected NettyMessagingService getNettyMessagingService() {
42 - return messagingService;
43 } 53 }
44 54
45 @Override 55 @Override
...@@ -49,38 +59,23 @@ public class NettyProtocolServer implements ProtocolServer { ...@@ -49,38 +59,23 @@ public class NettyProtocolServer implements ProtocolServer {
49 59
50 @Override 60 @Override
51 public CompletableFuture<Void> listen() { 61 public CompletableFuture<Void> listen() {
52 - try { 62 + return CompletableFuture.completedFuture(null);
53 - messagingService.activate();
54 - return CompletableFuture.completedFuture(null);
55 - } catch (Exception e) {
56 - CompletableFuture<Void> future = new CompletableFuture<>();
57 - future.completeExceptionally(e);
58 - return future;
59 - }
60 } 63 }
61 64
62 @Override 65 @Override
63 public CompletableFuture<Void> close() { 66 public CompletableFuture<Void> close() {
64 - CompletableFuture<Void> future = new CompletableFuture<>(); 67 + return CompletableFuture.completedFuture(null);
65 - try {
66 - messagingService.deactivate();
67 - future.complete(null);
68 - return future;
69 - } catch (Exception e) {
70 - future.completeExceptionally(e);
71 - return future;
72 - }
73 } 68 }
74 69
75 - private class CopycatMessageHandler<T> implements MessageHandler { 70 + private class CopycatMessageHandler<T> implements ClusterMessageHandler {
76 71
77 @Override 72 @Override
78 - public void handle(Message message) throws IOException { 73 + public void handle(ClusterMessage message) {
79 - T request = NettyProtocol.SERIALIZER.decode(message.payload()); 74 + T request = ClusterMessagingProtocol.SERIALIZER.decode(message.payload());
80 if (request.getClass().equals(PingRequest.class)) { 75 if (request.getClass().equals(PingRequest.class)) {
81 handler.ping((PingRequest) request).whenComplete((response, error) -> { 76 handler.ping((PingRequest) request).whenComplete((response, error) -> {
82 try { 77 try {
83 - message.respond(NettyProtocol.SERIALIZER.encode(response)); 78 + message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
84 } catch (Exception e) { 79 } catch (Exception e) {
85 log.error("Failed to respond to ping request", e); 80 log.error("Failed to respond to ping request", e);
86 } 81 }
...@@ -88,7 +83,7 @@ public class NettyProtocolServer implements ProtocolServer { ...@@ -88,7 +83,7 @@ public class NettyProtocolServer implements ProtocolServer {
88 } else if (request.getClass().equals(PollRequest.class)) { 83 } else if (request.getClass().equals(PollRequest.class)) {
89 handler.poll((PollRequest) request).whenComplete((response, error) -> { 84 handler.poll((PollRequest) request).whenComplete((response, error) -> {
90 try { 85 try {
91 - message.respond(NettyProtocol.SERIALIZER.encode(response)); 86 + message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
92 } catch (Exception e) { 87 } catch (Exception e) {
93 log.error("Failed to respond to poll request", e); 88 log.error("Failed to respond to poll request", e);
94 } 89 }
...@@ -96,7 +91,7 @@ public class NettyProtocolServer implements ProtocolServer { ...@@ -96,7 +91,7 @@ public class NettyProtocolServer implements ProtocolServer {
96 } else if (request.getClass().equals(SyncRequest.class)) { 91 } else if (request.getClass().equals(SyncRequest.class)) {
97 handler.sync((SyncRequest) request).whenComplete((response, error) -> { 92 handler.sync((SyncRequest) request).whenComplete((response, error) -> {
98 try { 93 try {
99 - message.respond(NettyProtocol.SERIALIZER.encode(response)); 94 + message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
100 } catch (Exception e) { 95 } catch (Exception e) {
101 log.error("Failed to respond to sync request", e); 96 log.error("Failed to respond to sync request", e);
102 } 97 }
...@@ -104,7 +99,7 @@ public class NettyProtocolServer implements ProtocolServer { ...@@ -104,7 +99,7 @@ public class NettyProtocolServer implements ProtocolServer {
104 } else if (request.getClass().equals(SubmitRequest.class)) { 99 } else if (request.getClass().equals(SubmitRequest.class)) {
105 handler.submit((SubmitRequest) request).whenComplete((response, error) -> { 100 handler.submit((SubmitRequest) request).whenComplete((response, error) -> {
106 try { 101 try {
107 - message.respond(NettyProtocol.SERIALIZER.encode(response)); 102 + message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
108 } catch (Exception e) { 103 } catch (Exception e) {
109 log.error("Failed to respond to submit request", e); 104 log.error("Failed to respond to submit request", e);
110 } 105 }
...@@ -112,4 +107,4 @@ public class NettyProtocolServer implements ProtocolServer { ...@@ -112,4 +107,4 @@ public class NettyProtocolServer implements ProtocolServer {
112 } 107 }
113 } 108 }
114 } 109 }
115 -} 110 +}
...\ No newline at end of file ...\ No newline at end of file
......
...@@ -11,37 +11,22 @@ import net.kuujo.copycat.protocol.SubmitRequest; ...@@ -11,37 +11,22 @@ import net.kuujo.copycat.protocol.SubmitRequest;
11 import net.kuujo.copycat.protocol.SubmitResponse; 11 import net.kuujo.copycat.protocol.SubmitResponse;
12 import net.kuujo.copycat.spi.protocol.ProtocolClient; 12 import net.kuujo.copycat.spi.protocol.ProtocolClient;
13 13
14 -import org.apache.commons.lang3.RandomUtils;
15 -import org.onlab.netty.Endpoint;
16 -import org.onlab.netty.NettyMessagingService;
17 import org.onlab.onos.store.service.DatabaseException; 14 import org.onlab.onos.store.service.DatabaseException;
18 import org.onlab.onos.store.service.ReadRequest; 15 import org.onlab.onos.store.service.ReadRequest;
19 import org.onlab.onos.store.service.WriteRequest; 16 import org.onlab.onos.store.service.WriteRequest;
20 17
21 public class DatabaseClient { 18 public class DatabaseClient {
22 19
23 - private final Endpoint copycatEp; 20 + private final ProtocolClient client;
24 - ProtocolClient client;
25 - NettyMessagingService messagingService;
26 21
27 - public DatabaseClient(Endpoint copycatEp) { 22 + public DatabaseClient(ProtocolClient client) {
28 - this.copycatEp = copycatEp; 23 + this.client = client;
29 } 24 }
30 25
31 private static String nextId() { 26 private static String nextId() {
32 return UUID.randomUUID().toString(); 27 return UUID.randomUUID().toString();
33 } 28 }
34 29
35 - public void activate() throws Exception {
36 - messagingService = new NettyMessagingService(RandomUtils.nextInt(10000, 40000));
37 - messagingService.activate();
38 - client = new NettyProtocolClient(copycatEp, messagingService);
39 - }
40 -
41 - public void deactivate() throws Exception {
42 - messagingService.deactivate();
43 - }
44 -
45 public boolean createTable(String tableName) { 30 public boolean createTable(String tableName) {
46 31
47 SubmitRequest request = 32 SubmitRequest request =
......
...@@ -18,7 +18,6 @@ import org.apache.felix.scr.annotations.Activate; ...@@ -18,7 +18,6 @@ import org.apache.felix.scr.annotations.Activate;
18 import org.apache.felix.scr.annotations.Deactivate; 18 import org.apache.felix.scr.annotations.Deactivate;
19 import org.apache.felix.scr.annotations.Reference; 19 import org.apache.felix.scr.annotations.Reference;
20 import org.apache.felix.scr.annotations.ReferenceCardinality; 20 import org.apache.felix.scr.annotations.ReferenceCardinality;
21 -import org.onlab.netty.Endpoint;
22 import org.onlab.onos.cluster.ClusterService; 21 import org.onlab.onos.cluster.ClusterService;
23 import org.onlab.onos.cluster.ControllerNode; 22 import org.onlab.onos.cluster.ControllerNode;
24 import org.onlab.onos.store.service.DatabaseAdminService; 23 import org.onlab.onos.store.service.DatabaseAdminService;
...@@ -50,6 +49,9 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService { ...@@ -50,6 +49,9 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
50 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY) 49 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
51 ClusterService clusterService; 50 ClusterService clusterService;
52 51
52 + @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
53 + ClusterMessagingProtocol copycatMessagingProtocol;
54 +
53 public static final String LOG_FILE_PREFIX = "onos-copy-cat-log"; 55 public static final String LOG_FILE_PREFIX = "onos-copy-cat-log";
54 56
55 private Copycat copycat; 57 private Copycat copycat;
...@@ -57,15 +59,14 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService { ...@@ -57,15 +59,14 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
57 59
58 @Activate 60 @Activate
59 public void activate() { 61 public void activate() {
60 - // FIXME hack tcpPort +1 for copycat communication
61 TcpMember localMember = 62 TcpMember localMember =
62 new TcpMember( 63 new TcpMember(
63 clusterService.getLocalNode().ip().toString(), 64 clusterService.getLocalNode().ip().toString(),
64 - clusterService.getLocalNode().tcpPort() + 1); 65 + clusterService.getLocalNode().tcpPort());
65 List<TcpMember> remoteMembers = Lists.newArrayList(); 66 List<TcpMember> remoteMembers = Lists.newArrayList();
66 67
67 for (ControllerNode node : clusterService.getNodes()) { 68 for (ControllerNode node : clusterService.getNodes()) {
68 - TcpMember member = new TcpMember(node.ip().toString(), node.tcpPort() + 1); 69 + TcpMember member = new TcpMember(node.ip().toString(), node.tcpPort());
69 if (!member.equals(localMember)) { 70 if (!member.equals(localMember)) {
70 remoteMembers.add(member); 71 remoteMembers.add(member);
71 } 72 }
...@@ -84,10 +85,10 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService { ...@@ -84,10 +85,10 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
84 ControllerNode thisNode = clusterService.getLocalNode(); 85 ControllerNode thisNode = clusterService.getLocalNode();
85 Log consensusLog = new ChronicleLog(LOG_FILE_PREFIX + "_" + thisNode.id()); 86 Log consensusLog = new ChronicleLog(LOG_FILE_PREFIX + "_" + thisNode.id());
86 87
87 - copycat = new Copycat(stateMachine, consensusLog, cluster, new NettyProtocol()); 88 + copycat = new Copycat(stateMachine, consensusLog, cluster, copycatMessagingProtocol);
88 copycat.start(); 89 copycat.start();
89 90
90 - client = new DatabaseClient(new Endpoint(localMember.host(), localMember.port())); 91 + client = new DatabaseClient(copycatMessagingProtocol.createClient(localMember));
91 92
92 log.info("Started."); 93 log.info("Started.");
93 } 94 }
......
...@@ -27,7 +27,7 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -27,7 +27,7 @@ public class DatabaseStateMachine implements StateMachine {
27 serializerPool = KryoNamespace.newBuilder() 27 serializerPool = KryoNamespace.newBuilder()
28 .register(VersionedValue.class) 28 .register(VersionedValue.class)
29 .register(State.class) 29 .register(State.class)
30 - .register(NettyProtocol.COMMON) 30 + .register(ClusterMessagingProtocol.COMMON)
31 .build() 31 .build()
32 .populate(1); 32 .populate(1);
33 } 33 }
......