package org.apache.cassandra.net;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.channels.SocketChannel;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.metrics.ConnectionMetrics;
import org.apache.cassandra.security.SSLFactory;
import org.apache.cassandra.utils.FBUtilities;
public class OutboundTcpConnectionPool
{
public static final long LARGE_MESSAGE_THRESHOLD =
Long.getLong(Config.PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64);
private final InetAddress id;
private final CountDownLatch started;
public final OutboundTcpConnection smallMessages;
public final OutboundTcpConnection largeMessages;
public final OutboundTcpConnection gossipMessages;
private InetAddress resetEndpoint;
private ConnectionMetrics metrics;
private final BackPressureState backPressureState;
OutboundTcpConnectionPool(InetAddress remoteEp, BackPressureState backPressureState)
{
id = remoteEp;
resetEndpoint = SystemKeyspace.getPreferredIP(remoteEp);
started = new CountDownLatch(1);
smallMessages = new OutboundTcpConnection(this, "Small");
largeMessages = new OutboundTcpConnection(this, "Large");
gossipMessages = new OutboundTcpConnection(this, "Gossip");
this.backPressureState = backPressureState;
}
OutboundTcpConnection getConnection(MessageOut msg)
{
if (Stage.GOSSIP == msg.getStage())
return gossipMessages;
return msg.payloadSize(smallMessages.getTargetVersion()) > LARGE_MESSAGE_THRESHOLD
? largeMessages
: smallMessages;
}
public BackPressureState getBackPressureState()
{
return backPressureState;
}
void reset()
{
for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
conn.closeSocket(false);
}
public void resetToNewerVersion(int version)
{
for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
{
if (version > conn.getTargetVersion())
conn.softCloseSocket();
}
}
public void reset(InetAddress remoteEP)
{
SystemKeyspace.updatePreferredIP(id, remoteEP);
resetEndpoint = remoteEP;
for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
conn.softCloseSocket();
metrics.release();
metrics = new ConnectionMetrics(resetEndpoint, this);
}
public long getTimeouts()
{
return metrics.timeouts.getCount();
}
public void incrementTimeout()
{
metrics.timeouts.mark();
}
public Socket newSocket() throws IOException
{
return newSocket(endPoint());
}
@SuppressWarnings("resource")
public static Socket newSocket(InetAddress endpoint) throws IOException
{
if (isEncryptedChannel(endpoint))
{
return SSLFactory.getSocket(DatabaseDescriptor.getServerEncryptionOptions(), endpoint, DatabaseDescriptor.getSSLStoragePort());
}
else
{
SocketChannel channel = SocketChannel.open();
channel.connect(new InetSocketAddress(endpoint, DatabaseDescriptor.getStoragePort()));
return channel.socket();
}
}
public InetAddress endPoint()
{
if (id.equals(FBUtilities.getBroadcastAddress()))
return FBUtilities.getLocalAddress();
return resetEndpoint;
}
public static boolean isEncryptedChannel(InetAddress address)
{
IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
switch (DatabaseDescriptor.getServerEncryptionOptions().internode_encryption)
{
case none:
return false;
case all:
break;
case dc:
if (snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddress())))
return false;
break;
case rack:
if (snitch.getRack(address).equals(snitch.getRack(FBUtilities.getBroadcastAddress()))
&& snitch.getDatacenter(address).equals(snitch.getDatacenter(FBUtilities.getBroadcastAddress())))
return false;
break;
}
return true;
}
public void start()
{
smallMessages.start();
largeMessages.start();
gossipMessages.start();
metrics = new ConnectionMetrics(id, this);
started.countDown();
}
public void waitForStarted()
{
if (started.getCount() == 0)
return;
boolean error = false;
try
{
if (!started.await(1, TimeUnit.MINUTES))
error = true;
}
catch (InterruptedException e)
{
Thread.currentThread().interrupt();
error = true;
}
if (error)
throw new IllegalStateException(String.format("Connections to %s are not started!", id.getHostAddress()));
}
public void close()
{
if (largeMessages != null)
largeMessages.closeSocket(true);
if (smallMessages != null)
smallMessages.closeSocket(true);
if (gossipMessages != null)
gossipMessages.closeSocket(true);
metrics.release();
}
}