Copycat messaging new happens over the same cluster messaging used for all other ONOS p2p communication

This commit is contained in:
Madan Jampani 2014-11-04 21:37:13 -08:00
parent 657626eb45
commit 9b19a82ced
10 changed files with 254 additions and 206 deletions

View File

@ -26,7 +26,7 @@ public interface DatabaseService {
/** /**
* Performs a write operation on the database. * Performs a write operation on the database.
* @param request * @param request write request
* @return write result. * @return write result.
* @throws DatabaseException if there is failure in execution write. * @throws DatabaseException if there is failure in execution write.
*/ */

View File

@ -17,15 +17,15 @@ public class ReadResult {
} }
/** /**
* Database table name. * Returns database table name.
* @return * @return table name.
*/ */
public String tableName() { public String tableName() {
return tableName; return tableName;
} }
/** /**
* Database table key. * Returns database table key.
* @return key. * @return key.
*/ */
public String key() { public String key() {
@ -33,7 +33,7 @@ public class ReadResult {
} }
/** /**
* value associated with the key. * Returns value associated with the key.
* @return non-null value if the table contains one, null otherwise. * @return non-null value if the table contains one, null otherwise.
*/ */
public VersionedValue value() { public VersionedValue value() {

View File

@ -12,8 +12,8 @@ public class VersionedValue {
/** /**
* Creates a new instance with the specified value and version. * Creates a new instance with the specified value and version.
* @param value * @param value value
* @param version * @param version version
*/ */
public VersionedValue(byte[] value, long version) { public VersionedValue(byte[] value, long version) {
this.value = value; this.value = value;

View File

@ -1,5 +1,8 @@
package org.onlab.onos.store.service.impl; package org.onlab.onos.store.service.impl;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.slf4j.LoggerFactory.getLogger;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
@ -27,6 +30,16 @@ import net.kuujo.copycat.spi.protocol.Protocol;
import net.kuujo.copycat.spi.protocol.ProtocolClient; import net.kuujo.copycat.spi.protocol.ProtocolClient;
import net.kuujo.copycat.spi.protocol.ProtocolServer; import net.kuujo.copycat.spi.protocol.ProtocolServer;
import org.apache.felix.scr.annotations.Activate;
import org.apache.felix.scr.annotations.Component;
import org.apache.felix.scr.annotations.Deactivate;
import org.apache.felix.scr.annotations.Reference;
import org.apache.felix.scr.annotations.ReferenceCardinality;
import org.apache.felix.scr.annotations.Service;
import org.onlab.onos.cluster.ClusterService;
import org.onlab.onos.cluster.ControllerNode;
import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
import org.onlab.onos.store.cluster.messaging.MessageSubject;
import org.onlab.onos.store.serializers.ImmutableListSerializer; import org.onlab.onos.store.serializers.ImmutableListSerializer;
import org.onlab.onos.store.serializers.ImmutableMapSerializer; import org.onlab.onos.store.serializers.ImmutableMapSerializer;
import org.onlab.onos.store.serializers.ImmutableSetSerializer; import org.onlab.onos.store.serializers.ImmutableSetSerializer;
@ -37,6 +50,7 @@ import org.onlab.onos.store.service.VersionedValue;
import org.onlab.onos.store.service.WriteRequest; import org.onlab.onos.store.service.WriteRequest;
import org.onlab.onos.store.service.WriteResult; import org.onlab.onos.store.service.WriteResult;
import org.onlab.util.KryoNamespace; import org.onlab.util.KryoNamespace;
import org.slf4j.Logger;
import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Input;
@ -46,17 +60,44 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
/** /**
* {@link Protocol} based on {@link org.onlab.netty.NettyMessagingService}. * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under
* the License.
*/ */
public class NettyProtocol implements Protocol<TcpMember> {
public static final String COPYCAT_PING = "copycat-raft-consensus-ping"; @Component(immediate = true)
public static final String COPYCAT_SYNC = "copycat-raft-consensus-sync"; @Service
public static final String COPYCAT_POLL = "copycat-raft-consensus-poll"; public class ClusterMessagingProtocol implements Protocol<TcpMember> {
public static final String COPYCAT_SUBMIT = "copycat-raft-consensus-submit";
// TODO: make this configurable. private final Logger log = getLogger(getClass());
public static final long RETRY_INTERVAL_MILLIS = 2000;
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
ClusterService clusterService;
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
ClusterCommunicationService clusterCommunicator;
public static final MessageSubject COPYCAT_PING =
new MessageSubject("copycat-raft-consensus-ping");
public static final MessageSubject COPYCAT_SYNC =
new MessageSubject("copycat-raft-consensus-sync");
public static final MessageSubject COPYCAT_POLL =
new MessageSubject("copycat-raft-consensus-poll");
public static final MessageSubject COPYCAT_SUBMIT =
new MessageSubject("copycat-raft-consensus-submit");
private static final KryoNamespace COPYCAT = KryoNamespace.newBuilder() private static final KryoNamespace COPYCAT = KryoNamespace.newBuilder()
.register(PingRequest.class) .register(PingRequest.class)
@ -76,8 +117,7 @@ public class NettyProtocol implements Protocol<TcpMember> {
.register(TcpMember.class) .register(TcpMember.class)
.build(); .build();
// TODO: Move to the right place. private static final KryoNamespace DATABASE = KryoNamespace.newBuilder()
private static final KryoNamespace CRAFT = KryoNamespace.newBuilder()
.register(ReadRequest.class) .register(ReadRequest.class)
.register(WriteRequest.class) .register(WriteRequest.class)
.register(InternalReadResult.class) .register(InternalReadResult.class)
@ -116,31 +156,41 @@ public class NettyProtocol implements Protocol<TcpMember> {
serializerPool = KryoNamespace.newBuilder() serializerPool = KryoNamespace.newBuilder()
.register(COPYCAT) .register(COPYCAT)
.register(COMMON) .register(COMMON)
.register(CRAFT) .register(DATABASE)
.build() .build()
.populate(1); .populate(1);
} }
}; };
private NettyProtocolServer server = null; @Activate
public void activate() {
log.info("Started.");
}
// FIXME: This is a total hack.Assumes @Deactivate
// ProtocolServer is initialized before ProtocolClient public void deactivate() {
protected NettyProtocolServer getServer() { log.info("Stopped.");
if (server == null) {
throw new IllegalStateException("ProtocolServer is not initialized yet!");
}
return server;
} }
@Override @Override
public ProtocolServer createServer(TcpMember member) { public ProtocolServer createServer(TcpMember member) {
server = new NettyProtocolServer(member); return new ClusterMessagingProtocolServer(clusterCommunicator);
return server;
} }
@Override @Override
public ProtocolClient createClient(TcpMember member) { public ProtocolClient createClient(TcpMember member) {
return new NettyProtocolClient(this, member); ControllerNode node = getControllerNode(member.host(), member.port());
checkNotNull(node, "A valid controller node is expected");
return new ClusterMessagingProtocolClient(
clusterCommunicator, node);
} }
}
private ControllerNode getControllerNode(String host, int port) {
for (ControllerNode node : clusterService.getNodes()) {
if (node.ip().toString().equals(host) && node.tcpPort() == port) {
return node;
}
}
return null;
}
}

View File

@ -11,7 +11,6 @@ import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import net.kuujo.copycat.cluster.TcpMember;
import net.kuujo.copycat.protocol.PingRequest; import net.kuujo.copycat.protocol.PingRequest;
import net.kuujo.copycat.protocol.PingResponse; import net.kuujo.copycat.protocol.PingResponse;
import net.kuujo.copycat.protocol.PollRequest; import net.kuujo.copycat.protocol.PollRequest;
@ -22,37 +21,54 @@ import net.kuujo.copycat.protocol.SyncRequest;
import net.kuujo.copycat.protocol.SyncResponse; import net.kuujo.copycat.protocol.SyncResponse;
import net.kuujo.copycat.spi.protocol.ProtocolClient; import net.kuujo.copycat.spi.protocol.ProtocolClient;
import org.onlab.netty.Endpoint; import org.onlab.onos.cluster.ControllerNode;
import org.onlab.netty.NettyMessagingService; import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
import org.onlab.onos.store.cluster.messaging.ClusterMessage;
import org.onlab.onos.store.cluster.messaging.MessageSubject;
import org.slf4j.Logger; import org.slf4j.Logger;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
/** /**
* {@link NettyMessagingService} based Copycat protocol client. * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under
* the License.
*/ */
public class NettyProtocolClient implements ProtocolClient {
public class ClusterMessagingProtocolClient implements ProtocolClient {
private final Logger log = getLogger(getClass()); private final Logger log = getLogger(getClass());
private static final ThreadFactory THREAD_FACTORY = private static final ThreadFactory THREAD_FACTORY =
new ThreadFactoryBuilder().setNameFormat("copycat-netty-messaging-%d").build(); new ThreadFactoryBuilder().setNameFormat("copycat-netty-messaging-%d").build();
// Remote endpoint, this client instance is used public static final long RETRY_INTERVAL_MILLIS = 2000;
// for communicating with.
private final Endpoint remoteEp;
private final NettyMessagingService messagingService;
// TODO: Is 10 the right number of threads? private final ClusterCommunicationService clusterCommunicator;
private final ControllerNode remoteNode;
// FIXME: Thread pool sizing.
private static final ScheduledExecutorService THREAD_POOL = private static final ScheduledExecutorService THREAD_POOL =
new ScheduledThreadPoolExecutor(10, THREAD_FACTORY); new ScheduledThreadPoolExecutor(10, THREAD_FACTORY);
public NettyProtocolClient(NettyProtocol protocol, TcpMember member) { public ClusterMessagingProtocolClient(
this(new Endpoint(member.host(), member.port()), protocol.getServer().getNettyMessagingService()); ClusterCommunicationService clusterCommunicator,
} ControllerNode remoteNode) {
this.clusterCommunicator = clusterCommunicator;
public NettyProtocolClient(Endpoint remoteEp, NettyMessagingService messagingService) { this.remoteNode = remoteNode;
this.remoteEp = remoteEp;
this.messagingService = messagingService;
} }
@Override @Override
@ -85,16 +101,16 @@ public class NettyProtocolClient implements ProtocolClient {
return CompletableFuture.completedFuture(null); return CompletableFuture.completedFuture(null);
} }
public <I> String messageType(I input) { public <I> MessageSubject messageType(I input) {
Class<?> clazz = input.getClass(); Class<?> clazz = input.getClass();
if (clazz.equals(PollRequest.class)) { if (clazz.equals(PollRequest.class)) {
return NettyProtocol.COPYCAT_POLL; return ClusterMessagingProtocol.COPYCAT_POLL;
} else if (clazz.equals(SyncRequest.class)) { } else if (clazz.equals(SyncRequest.class)) {
return NettyProtocol.COPYCAT_SYNC; return ClusterMessagingProtocol.COPYCAT_SYNC;
} else if (clazz.equals(SubmitRequest.class)) { } else if (clazz.equals(SubmitRequest.class)) {
return NettyProtocol.COPYCAT_SUBMIT; return ClusterMessagingProtocol.COPYCAT_SUBMIT;
} else if (clazz.equals(PingRequest.class)) { } else if (clazz.equals(PingRequest.class)) {
return NettyProtocol.COPYCAT_PING; return ClusterMessagingProtocol.COPYCAT_PING;
} else { } else {
throw new IllegalArgumentException("Unknown class " + clazz.getName()); throw new IllegalArgumentException("Unknown class " + clazz.getName());
} }
@ -109,33 +125,34 @@ public class NettyProtocolClient implements ProtocolClient {
private class RPCTask<I, O> implements Runnable { private class RPCTask<I, O> implements Runnable {
private final String messageType; private final ClusterMessage message;
private final byte[] payload;
private final CompletableFuture<O> future; private final CompletableFuture<O> future;
public RPCTask(I request, CompletableFuture<O> future) { public RPCTask(I request, CompletableFuture<O> future) {
this.messageType = messageType(request); this.message =
this.payload = NettyProtocol.SERIALIZER.encode(request); new ClusterMessage(
null,
messageType(request),
ClusterMessagingProtocol.SERIALIZER.encode(request));
this.future = future; this.future = future;
} }
@Override @Override
public void run() { public void run() {
try { try {
byte[] response = messagingService byte[] response = clusterCommunicator
.sendAndReceive(remoteEp, messageType, payload) .sendAndReceive(message, remoteNode.id())
.get(NettyProtocol.RETRY_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); .get(RETRY_INTERVAL_MILLIS, TimeUnit.MILLISECONDS);
future.complete(NettyProtocol.SERIALIZER.decode(response)); future.complete(ClusterMessagingProtocol.SERIALIZER.decode(response));
} catch (IOException | InterruptedException | ExecutionException | TimeoutException e) { } catch (IOException | InterruptedException | ExecutionException | TimeoutException e) {
if (messageType.equals(NettyProtocol.COPYCAT_SYNC) || if (message.subject().equals(ClusterMessagingProtocol.COPYCAT_SYNC) ||
messageType.equals(NettyProtocol.COPYCAT_PING)) { message.subject().equals(ClusterMessagingProtocol.COPYCAT_PING)) {
log.warn("Request to {} failed. Will retry " log.warn("Request to {} failed. Will retry "
+ "in {} ms", remoteEp, NettyProtocol.RETRY_INTERVAL_MILLIS); + "in {} ms", remoteNode, RETRY_INTERVAL_MILLIS);
THREAD_POOL.schedule( THREAD_POOL.schedule(
this, this,
NettyProtocol.RETRY_INTERVAL_MILLIS, RETRY_INTERVAL_MILLIS,
TimeUnit.MILLISECONDS); TimeUnit.MILLISECONDS);
} else { } else {
future.completeExceptionally(e); future.completeExceptionally(e);
@ -145,4 +162,4 @@ public class NettyProtocolClient implements ProtocolClient {
} }
} }
} }
} }

View File

@ -0,0 +1,110 @@
package org.onlab.onos.store.service.impl;
import static org.slf4j.LoggerFactory.getLogger;
import java.util.concurrent.CompletableFuture;
import net.kuujo.copycat.protocol.PingRequest;
import net.kuujo.copycat.protocol.PollRequest;
import net.kuujo.copycat.protocol.RequestHandler;
import net.kuujo.copycat.protocol.SubmitRequest;
import net.kuujo.copycat.protocol.SyncRequest;
import net.kuujo.copycat.spi.protocol.ProtocolServer;
import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
import org.onlab.onos.store.cluster.messaging.ClusterMessage;
import org.onlab.onos.store.cluster.messaging.ClusterMessageHandler;
import org.slf4j.Logger;
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under
* the License.
*/
public class ClusterMessagingProtocolServer implements ProtocolServer {
private final Logger log = getLogger(getClass());
private RequestHandler handler;
public ClusterMessagingProtocolServer(ClusterCommunicationService clusterCommunicator) {
clusterCommunicator.addSubscriber(
ClusterMessagingProtocol.COPYCAT_PING, new CopycatMessageHandler<PingRequest>());
clusterCommunicator.addSubscriber(
ClusterMessagingProtocol.COPYCAT_SYNC, new CopycatMessageHandler<SyncRequest>());
clusterCommunicator.addSubscriber(
ClusterMessagingProtocol.COPYCAT_POLL, new CopycatMessageHandler<PollRequest>());
clusterCommunicator.addSubscriber(
ClusterMessagingProtocol.COPYCAT_SUBMIT, new CopycatMessageHandler<SubmitRequest>());
}
@Override
public void requestHandler(RequestHandler handler) {
this.handler = handler;
}
@Override
public CompletableFuture<Void> listen() {
return CompletableFuture.completedFuture(null);
}
@Override
public CompletableFuture<Void> close() {
return CompletableFuture.completedFuture(null);
}
private class CopycatMessageHandler<T> implements ClusterMessageHandler {
@Override
public void handle(ClusterMessage message) {
T request = ClusterMessagingProtocol.SERIALIZER.decode(message.payload());
if (request.getClass().equals(PingRequest.class)) {
handler.ping((PingRequest) request).whenComplete((response, error) -> {
try {
message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to ping request", e);
}
});
} else if (request.getClass().equals(PollRequest.class)) {
handler.poll((PollRequest) request).whenComplete((response, error) -> {
try {
message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to poll request", e);
}
});
} else if (request.getClass().equals(SyncRequest.class)) {
handler.sync((SyncRequest) request).whenComplete((response, error) -> {
try {
message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to sync request", e);
}
});
} else if (request.getClass().equals(SubmitRequest.class)) {
handler.submit((SubmitRequest) request).whenComplete((response, error) -> {
try {
message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to submit request", e);
}
});
}
}
}
}

View File

@ -11,37 +11,22 @@ import net.kuujo.copycat.protocol.SubmitRequest;
import net.kuujo.copycat.protocol.SubmitResponse; import net.kuujo.copycat.protocol.SubmitResponse;
import net.kuujo.copycat.spi.protocol.ProtocolClient; import net.kuujo.copycat.spi.protocol.ProtocolClient;
import org.apache.commons.lang3.RandomUtils;
import org.onlab.netty.Endpoint;
import org.onlab.netty.NettyMessagingService;
import org.onlab.onos.store.service.DatabaseException; import org.onlab.onos.store.service.DatabaseException;
import org.onlab.onos.store.service.ReadRequest; import org.onlab.onos.store.service.ReadRequest;
import org.onlab.onos.store.service.WriteRequest; import org.onlab.onos.store.service.WriteRequest;
public class DatabaseClient { public class DatabaseClient {
private final Endpoint copycatEp; private final ProtocolClient client;
ProtocolClient client;
NettyMessagingService messagingService;
public DatabaseClient(Endpoint copycatEp) { public DatabaseClient(ProtocolClient client) {
this.copycatEp = copycatEp; this.client = client;
} }
private static String nextId() { private static String nextId() {
return UUID.randomUUID().toString(); return UUID.randomUUID().toString();
} }
public void activate() throws Exception {
messagingService = new NettyMessagingService(RandomUtils.nextInt(10000, 40000));
messagingService.activate();
client = new NettyProtocolClient(copycatEp, messagingService);
}
public void deactivate() throws Exception {
messagingService.deactivate();
}
public boolean createTable(String tableName) { public boolean createTable(String tableName) {
SubmitRequest request = SubmitRequest request =

View File

@ -18,7 +18,6 @@ import org.apache.felix.scr.annotations.Activate;
import org.apache.felix.scr.annotations.Deactivate; import org.apache.felix.scr.annotations.Deactivate;
import org.apache.felix.scr.annotations.Reference; import org.apache.felix.scr.annotations.Reference;
import org.apache.felix.scr.annotations.ReferenceCardinality; import org.apache.felix.scr.annotations.ReferenceCardinality;
import org.onlab.netty.Endpoint;
import org.onlab.onos.cluster.ClusterService; import org.onlab.onos.cluster.ClusterService;
import org.onlab.onos.cluster.ControllerNode; import org.onlab.onos.cluster.ControllerNode;
import org.onlab.onos.store.service.DatabaseAdminService; import org.onlab.onos.store.service.DatabaseAdminService;
@ -50,6 +49,9 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY) @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
ClusterService clusterService; ClusterService clusterService;
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
ClusterMessagingProtocol copycatMessagingProtocol;
public static final String LOG_FILE_PREFIX = "onos-copy-cat-log"; public static final String LOG_FILE_PREFIX = "onos-copy-cat-log";
private Copycat copycat; private Copycat copycat;
@ -57,15 +59,14 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
@Activate @Activate
public void activate() { public void activate() {
// FIXME hack tcpPort +1 for copycat communication
TcpMember localMember = TcpMember localMember =
new TcpMember( new TcpMember(
clusterService.getLocalNode().ip().toString(), clusterService.getLocalNode().ip().toString(),
clusterService.getLocalNode().tcpPort() + 1); clusterService.getLocalNode().tcpPort());
List<TcpMember> remoteMembers = Lists.newArrayList(); List<TcpMember> remoteMembers = Lists.newArrayList();
for (ControllerNode node : clusterService.getNodes()) { for (ControllerNode node : clusterService.getNodes()) {
TcpMember member = new TcpMember(node.ip().toString(), node.tcpPort() + 1); TcpMember member = new TcpMember(node.ip().toString(), node.tcpPort());
if (!member.equals(localMember)) { if (!member.equals(localMember)) {
remoteMembers.add(member); remoteMembers.add(member);
} }
@ -84,10 +85,10 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
ControllerNode thisNode = clusterService.getLocalNode(); ControllerNode thisNode = clusterService.getLocalNode();
Log consensusLog = new ChronicleLog(LOG_FILE_PREFIX + "_" + thisNode.id()); Log consensusLog = new ChronicleLog(LOG_FILE_PREFIX + "_" + thisNode.id());
copycat = new Copycat(stateMachine, consensusLog, cluster, new NettyProtocol()); copycat = new Copycat(stateMachine, consensusLog, cluster, copycatMessagingProtocol);
copycat.start(); copycat.start();
client = new DatabaseClient(new Endpoint(localMember.host(), localMember.port())); client = new DatabaseClient(copycatMessagingProtocol.createClient(localMember));
log.info("Started."); log.info("Started.");
} }

View File

@ -27,7 +27,7 @@ public class DatabaseStateMachine implements StateMachine {
serializerPool = KryoNamespace.newBuilder() serializerPool = KryoNamespace.newBuilder()
.register(VersionedValue.class) .register(VersionedValue.class)
.register(State.class) .register(State.class)
.register(NettyProtocol.COMMON) .register(ClusterMessagingProtocol.COMMON)
.build() .build()
.populate(1); .populate(1);
} }

View File

@ -1,115 +0,0 @@
package org.onlab.onos.store.service.impl;
import static org.slf4j.LoggerFactory.getLogger;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import net.kuujo.copycat.cluster.TcpMember;
import net.kuujo.copycat.protocol.PingRequest;
import net.kuujo.copycat.protocol.PollRequest;
import net.kuujo.copycat.protocol.RequestHandler;
import net.kuujo.copycat.protocol.SubmitRequest;
import net.kuujo.copycat.protocol.SyncRequest;
import net.kuujo.copycat.spi.protocol.ProtocolServer;
import org.onlab.netty.Message;
import org.onlab.netty.MessageHandler;
import org.onlab.netty.NettyMessagingService;
import org.slf4j.Logger;
/**
* {@link NettyMessagingService} based Copycat protocol server.
*/
public class NettyProtocolServer implements ProtocolServer {
private final Logger log = getLogger(getClass());
private final NettyMessagingService messagingService;
private RequestHandler handler;
public NettyProtocolServer(TcpMember member) {
messagingService = new NettyMessagingService(member.host(), member.port());
messagingService.registerHandler(NettyProtocol.COPYCAT_PING, new CopycatMessageHandler<PingRequest>());
messagingService.registerHandler(NettyProtocol.COPYCAT_SYNC, new CopycatMessageHandler<SyncRequest>());
messagingService.registerHandler(NettyProtocol.COPYCAT_POLL, new CopycatMessageHandler<PollRequest>());
messagingService.registerHandler(NettyProtocol.COPYCAT_SUBMIT, new CopycatMessageHandler<SubmitRequest>());
}
protected NettyMessagingService getNettyMessagingService() {
return messagingService;
}
@Override
public void requestHandler(RequestHandler handler) {
this.handler = handler;
}
@Override
public CompletableFuture<Void> listen() {
try {
messagingService.activate();
return CompletableFuture.completedFuture(null);
} catch (Exception e) {
CompletableFuture<Void> future = new CompletableFuture<>();
future.completeExceptionally(e);
return future;
}
}
@Override
public CompletableFuture<Void> close() {
CompletableFuture<Void> future = new CompletableFuture<>();
try {
messagingService.deactivate();
future.complete(null);
return future;
} catch (Exception e) {
future.completeExceptionally(e);
return future;
}
}
private class CopycatMessageHandler<T> implements MessageHandler {
@Override
public void handle(Message message) throws IOException {
T request = NettyProtocol.SERIALIZER.decode(message.payload());
if (request.getClass().equals(PingRequest.class)) {
handler.ping((PingRequest) request).whenComplete((response, error) -> {
try {
message.respond(NettyProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to ping request", e);
}
});
} else if (request.getClass().equals(PollRequest.class)) {
handler.poll((PollRequest) request).whenComplete((response, error) -> {
try {
message.respond(NettyProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to poll request", e);
}
});
} else if (request.getClass().equals(SyncRequest.class)) {
handler.sync((SyncRequest) request).whenComplete((response, error) -> {
try {
message.respond(NettyProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to sync request", e);
}
});
} else if (request.getClass().equals(SubmitRequest.class)) {
handler.submit((SubmitRequest) request).whenComplete((response, error) -> {
try {
message.respond(NettyProtocol.SERIALIZER.encode(response));
} catch (Exception e) {
log.error("Failed to respond to submit request", e);
}
});
}
}
}
}