package org.apache.cassandra.thrift;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.service.CassandraDaemon;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.thrift.TProcessor;
import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.server.TServer;
import org.apache.thrift.transport.TFramedTransport;
import org.apache.thrift.transport.TTransportFactory;
public class ThriftServer implements CassandraDaemon.Server
{
private static final Logger logger = LoggerFactory.getLogger(ThriftServer.class);
protected final InetAddress address;
protected final int port;
protected final int backlog;
private volatile ThriftServerThread server;
public ThriftServer(InetAddress address, int port, int backlog)
{
this.address = address;
this.port = port;
this.backlog = backlog;
}
public void start()
{
if (server == null)
{
CassandraServer iface = getCassandraServer();
server = new ThriftServerThread(address, port, backlog, getProcessor(iface), getTransportFactory());
server.start();
}
}
public synchronized void stop()
{
if (server != null)
{
server.stopServer();
try
{
server.join();
}
catch (InterruptedException e)
{
logger.error("Interrupted while waiting thrift server to stop", e);
}
server = null;
}
}
public boolean isRunning()
{
return server != null;
}
protected CassandraServer getCassandraServer()
{
return new CassandraServer();
}
protected TProcessor getProcessor(CassandraServer server)
{
return new Cassandra.Processor<Cassandra.Iface>(server);
}
protected TTransportFactory getTransportFactory()
{
int tFramedTransportSize = DatabaseDescriptor.getThriftFramedTransportSize();
return new TFramedTransport.Factory(tFramedTransportSize);
}
private static class ThriftServerThread extends Thread
{
private final TServer serverEngine;
public ThriftServerThread(InetAddress listenAddr,
int listenPort,
int listenBacklog,
TProcessor processor,
TTransportFactory transportFactory)
{
logger.info("Binding thrift service to {}:{}", listenAddr, listenPort);
TServerFactory.Args args = new TServerFactory.Args();
args.tProtocolFactory = new TBinaryProtocol.Factory(true, true);
args.addr = new InetSocketAddress(listenAddr, listenPort);
args.listenBacklog = listenBacklog;
args.processor = processor;
args.keepAlive = DatabaseDescriptor.getRpcKeepAlive();
args.sendBufferSize = DatabaseDescriptor.getRpcSendBufferSize();
args.recvBufferSize = DatabaseDescriptor.getRpcRecvBufferSize();
args.inTransportFactory = transportFactory;
args.outTransportFactory = transportFactory;
serverEngine = new TServerCustomFactory(DatabaseDescriptor.getRpcServerType()).buildTServer(args);
}
public void run()
{
logger.info("Listening for thrift clients...");
serverEngine.serve();
}
public void stopServer()
{
logger.info("Stop listening to thrift clients");
serverEngine.stop();
}
}
public static final class ThriftServerType
{
public final static String SYNC = "sync";
public final static String ASYNC = "async";
public final static String HSHA = "hsha";
}
}