Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 14 additions & 0 deletions config/elasticsearch.yml
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,20 @@
#
# transport.tcp.compress: true

# Enable SSL/TLS encryption for all communication between nodes (disabled by default):
#
# transport.tcp.ssl: true

# Settings for SSL/TLS encryption, used when transport.tcp.ssl is set to true
#
# transport.tcp.ssl.keystore: /path/to/the/keystore
# transport.tcp.ssl.keystore_password: password
# transport.tcp.ssl.keystore_algorithm: SunX509
#
# transport.tcp.ssl.truststore: /path/to/the/truststore
# transport.tcp.ssl.truststore_password: password
# transport.tcp.ssl.truststore_algorithm: PKIX

# Set a custom port to listen for HTTP traffic:
#
# http.port: 9200
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch 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.
*/

package org.elasticsearch.transport;

/**
* @author Tanguy Leroux - tlrx.dev@gmail.com
*/
public class SSLTransportException extends TransportException {

public SSLTransportException(String message) {
super(message);
}

public SSLTransportException(String message, Throwable cause) {
super(message, cause);
}
}
115 changes: 74 additions & 41 deletions src/main/java/org/elasticsearch/transport/netty/NettyTransport.java
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
import org.elasticsearch.transport.netty.ssl.*;
import org.elasticsearch.transport.support.TransportStreams;
import org.jboss.netty.bootstrap.ClientBootstrap;
import org.jboss.netty.bootstrap.ServerBootstrap;
Expand Down Expand Up @@ -107,6 +108,14 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
final Boolean tcpKeepAlive;

final Boolean reuseAddress;

final boolean ssl;
final String sslKeyStore;
final String sslKeyStorePassword;
final String sslKeyStoreAlgorithm;
final String sslTrustStore;
final String sslTrustStorePassword;
final String sslTrustStoreAlgorithm;

final ByteSizeValue tcpSendBufferSize;
final ByteSizeValue tcpReceiveBufferSize;
Expand Down Expand Up @@ -177,8 +186,16 @@ public NettyTransport(Settings settings, ThreadPool threadPool, NetworkService n
this.maxCumulationBufferCapacity = componentSettings.getAsBytesSize("max_cumulation_buffer_capacity", null);
this.maxCompositeBufferComponents = componentSettings.getAsInt("max_composite_buffer_components", -1);

logger.debug("using worker_count[{}], port[{}], bind_host[{}], publish_host[{}], compress[{}], connect_timeout[{}], connections_per_node[{}/{}/{}]",
workerCount, port, bindHost, publishHost, compress, connectTimeout, connectionsPerNodeLow, connectionsPerNodeMed, connectionsPerNodeHigh);
this.ssl = settings.getAsBoolean("transport.tcp.ssl", false);
this.sslKeyStore = settings.get("transport.tcp.ssl.keystore", System.getProperty("javax.net.ssl.keyStore"));
this.sslKeyStorePassword = settings.get("transport.tcp.ssl.keystore_password", System.getProperty("javax.net.ssl.keyStorePassword"));
this.sslKeyStoreAlgorithm = settings.get("transport.tcp.ssl.keystore_algorithm", System.getProperty("ssl.KeyManagerFactory.algorithm"));
this.sslTrustStore = settings.get("transport.tcp.ssl.truststore", System.getProperty("javax.net.ssl.trustStore"));
this.sslTrustStorePassword = settings.get("transport.tcp.ssl.truststore_password", System.getProperty("javax.net.ssl.trustStorePassword"));
this.sslTrustStoreAlgorithm = settings.get("ransport.tcp.ssl.truststore_algorithm", System.getProperty("ssl.TrustManagerFactory.algorithm"));

logger.debug("using worker_count[{}], port[{}], bind_host[{}], publish_host[{}], compress[{}], connect_timeout[{}], connections_per_node[{}/{}/{}], ssl[{}]",
workerCount, port, bindHost, publishHost, compress, connectTimeout, connectionsPerNodeLow, connectionsPerNodeMed, connectionsPerNodeHigh, ssl);
}

public Settings settings() {
Expand Down Expand Up @@ -208,26 +225,34 @@ protected void doStart() throws ElasticSearchException {
Executors.newCachedThreadPool(daemonThreadFactory(settings, "transport_client_worker")),
workerCount));
}
ChannelPipelineFactory clientPipelineFactory = new ChannelPipelineFactory() {
@Override
public ChannelPipeline getPipeline() throws Exception {
ChannelPipeline pipeline = Channels.pipeline();
SizeHeaderFrameDecoder sizeHeader = new SizeHeaderFrameDecoder();
if (maxCumulationBufferCapacity != null) {
if (maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
sizeHeader.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
sizeHeader.setMaxCumulationBufferCapacity((int) maxCumulationBufferCapacity.bytes());
ChannelPipelineFactory clientPipelineFactory = null;
if (ssl) {
clientPipelineFactory = new SecureClientChannelPipelineFactory(new SecureMessageChannelHandler(NettyTransport.this, logger),
sslKeyStore, sslKeyStorePassword, sslKeyStoreAlgorithm,
sslTrustStore, sslTrustStorePassword, sslTrustStoreAlgorithm,
maxCumulationBufferCapacity, maxCompositeBufferComponents);
} else {
clientPipelineFactory = new ChannelPipelineFactory() {
@Override
public ChannelPipeline getPipeline() throws Exception {
ChannelPipeline pipeline = Channels.pipeline();
SizeHeaderFrameDecoder sizeHeader = new SizeHeaderFrameDecoder();
if (maxCumulationBufferCapacity != null) {
if (maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
sizeHeader.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
sizeHeader.setMaxCumulationBufferCapacity((int) maxCumulationBufferCapacity.bytes());
}
}
if (maxCompositeBufferComponents != -1) {
sizeHeader.setMaxCumulationBufferComponents(maxCompositeBufferComponents);
}
pipeline.addLast("size", sizeHeader);
pipeline.addLast("dispatcher", new MessageChannelHandler(NettyTransport.this, logger));
return pipeline;
}
if (maxCompositeBufferComponents != -1) {
sizeHeader.setMaxCumulationBufferComponents(maxCompositeBufferComponents);
}
pipeline.addLast("size", sizeHeader);
pipeline.addLast("dispatcher", new MessageChannelHandler(NettyTransport.this, logger));
return pipeline;
}
};
};
}
clientBootstrap.setPipelineFactory(clientPipelineFactory);
clientBootstrap.setOption("connectTimeoutMillis", connectTimeout.millis());
if (tcpNoDelay != null) {
Expand Down Expand Up @@ -262,27 +287,35 @@ public ChannelPipeline getPipeline() throws Exception {
Executors.newCachedThreadPool(daemonThreadFactory(settings, "transport_server_worker")),
workerCount));
}
ChannelPipelineFactory serverPipelineFactory = new ChannelPipelineFactory() {
@Override
public ChannelPipeline getPipeline() throws Exception {
ChannelPipeline pipeline = Channels.pipeline();
pipeline.addLast("openChannels", serverOpenChannels);
SizeHeaderFrameDecoder sizeHeader = new SizeHeaderFrameDecoder();
if (maxCumulationBufferCapacity != null) {
if (maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
sizeHeader.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
sizeHeader.setMaxCumulationBufferCapacity((int) maxCumulationBufferCapacity.bytes());
}
}
if (maxCompositeBufferComponents != -1) {
sizeHeader.setMaxCumulationBufferComponents(maxCompositeBufferComponents);
}
pipeline.addLast("size", sizeHeader);
pipeline.addLast("dispatcher", new MessageChannelHandler(NettyTransport.this, logger));
return pipeline;
}
};
ChannelPipelineFactory serverPipelineFactory = null;
if (ssl) {
serverPipelineFactory = serverPipelineFactory = new SecureServerChannelPipelineFactory(new SecureMessageChannelHandler(NettyTransport.this, logger), serverOpenChannels,
sslKeyStore, sslKeyStorePassword, sslKeyStoreAlgorithm,
sslTrustStore, sslTrustStorePassword, sslTrustStoreAlgorithm,
maxCumulationBufferCapacity, maxCompositeBufferComponents);
} else {
serverPipelineFactory = new ChannelPipelineFactory() {
@Override
public ChannelPipeline getPipeline() throws Exception {
ChannelPipeline pipeline = Channels.pipeline();
pipeline.addLast("openChannels", serverOpenChannels);
SizeHeaderFrameDecoder sizeHeader = new SizeHeaderFrameDecoder();
if (maxCumulationBufferCapacity != null) {
if (maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
sizeHeader.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
sizeHeader.setMaxCumulationBufferCapacity((int) maxCumulationBufferCapacity.bytes());
}
}
if (maxCompositeBufferComponents != -1) {
sizeHeader.setMaxCumulationBufferComponents(maxCompositeBufferComponents);
}
pipeline.addLast("size", sizeHeader);
pipeline.addLast("dispatcher", new MessageChannelHandler(NettyTransport.this, logger));
return pipeline;
}
};
}
serverBootstrap.setPipelineFactory(serverPipelineFactory);
if (tcpNoDelay != null) {
serverBootstrap.setOption("child.tcpNoDelay", tcpNoDelay);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
package org.elasticsearch.transport.netty.ssl;

import java.io.FileInputStream;
import java.security.KeyStore;

import javax.net.ssl.KeyManagerFactory;
import javax.net.ssl.SSLContext;
import javax.net.ssl.TrustManager;
import javax.net.ssl.TrustManagerFactory;

import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.jboss.netty.channel.ChannelPipelineFactory;

/**
* ChannelPipelineFactory used for Client/Server SSL channel pipelines
*
* @author Tanguy Leroux
*
*/
public abstract class SSLChannelPipelineFactory implements ChannelPipelineFactory {

private static final ESLogger logger = Loggers.getLogger(SSLChannelPipelineFactory.class);

private SSLContext sslContext;

SecureMessageChannelHandler messageChannelHandler;

private String keyStore;

private String keyStorePassword;

private String keyStoreAlgorithm;

private String trustStore;

private String trustStorePassword;

private String trustStoreAlgorithm;

ByteSizeValue maxCumulationBufferCapacity;

int maxCompositeBufferComponents;

public SSLChannelPipelineFactory(SecureMessageChannelHandler channelHandler,
String sslKeyStore, String sslKeyStorePassword, String sslKeyStoreAlgorithm,
String sslTrustStore, String sslTrustStorePassword, String sslTrustStoreAlgorithm,
ByteSizeValue mCumulationBufferCapacity, int mCompositeBufferComponents) {

messageChannelHandler = channelHandler;
maxCumulationBufferCapacity = mCumulationBufferCapacity;
maxCompositeBufferComponents = mCompositeBufferComponents;

keyStore = sslKeyStore;
keyStorePassword = sslKeyStorePassword;
if (sslKeyStoreAlgorithm != null) {
keyStoreAlgorithm = sslKeyStoreAlgorithm;
} else {
keyStoreAlgorithm = KeyManagerFactory.getDefaultAlgorithm();
}

trustStore = sslTrustStore;
trustStorePassword = sslTrustStorePassword;
if (sslTrustStoreAlgorithm != null) {
trustStoreAlgorithm = sslTrustStoreAlgorithm;
} else {
trustStoreAlgorithm = TrustManagerFactory.getDefaultAlgorithm();
}

logger.debug("using keyStore[{}], keyAlgorithm[{}], trustStore[{}], trustAlgorithm[{}]", keyStore, keyStoreAlgorithm, trustStore, trustStoreAlgorithm);

KeyStore ks = null;
KeyManagerFactory kmf = null;
FileInputStream in = null;
try {
// Load KeyStore
ks = KeyStore.getInstance("jks");
in = new FileInputStream(keyStore);
ks.load(in, keyStorePassword.toCharArray());

// Initialize KeyManagerFactory
kmf = KeyManagerFactory.getInstance(keyStoreAlgorithm);
kmf.init(ks, keyStorePassword.toCharArray());
} catch (Exception e) {
throw new Error("Failed to initialize a KeyManagerFactory", e);
} finally {
try {
in.close();
} catch (Exception e2) {
}
}

TrustManager[] trustManagers = null;
try {
// Load TrustStore
in = new FileInputStream(trustStore);
ks.load(in, trustStorePassword.toCharArray());

// Initialize a trust manager factory with the trusted store
TrustManagerFactory trustFactory = TrustManagerFactory.getInstance(trustStoreAlgorithm);
trustFactory.init(ks);

// Retrieve the trust managers from the factory
trustManagers = trustFactory.getTrustManagers();
} catch (Exception e) {
throw new Error("Failed to initialize a TrustManagerFactory", e);
} finally {
try {
in.close();
} catch (Exception e2) {
}
}

// Initialize sslContext
try {
sslContext = SSLContext.getInstance("TLS");
sslContext.init(kmf.getKeyManagers(), trustManagers, null);
} catch (Exception e) {
throw new Error("Failed to initialize the SSLContext", e);
}
}

public SSLContext getSslContext() {
return sslContext;
}

public void setSslContext(SSLContext sslContext) {
this.sslContext = sslContext;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
package org.elasticsearch.transport.netty.ssl;

import javax.net.ssl.SSLEngine;

import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.transport.netty.SizeHeaderFrameDecoder;
import org.jboss.netty.channel.ChannelPipeline;
import org.jboss.netty.channel.Channels;
import org.jboss.netty.handler.ssl.SslHandler;

/**
* ChannelPipelineFactory used for Client SSL channel pipelines
*
* @author Tanguy Leroux
*
*/
public class SecureClientChannelPipelineFactory extends SSLChannelPipelineFactory {

public SecureClientChannelPipelineFactory(SecureMessageChannelHandler channelHandler,
String sslKeyStore, String sslKeyStorePassword, String sslKeyStoreAlgorithm,
String sslTrustStore, String sslTrustStorePassword, String sslTrustStoreAlgorithm,
ByteSizeValue maxCumulationBufferCapacity, int maxCompositeBufferComponents) {
super(channelHandler,
sslKeyStore, sslKeyStorePassword, sslKeyStoreAlgorithm,
sslTrustStore, sslTrustStorePassword, sslTrustStoreAlgorithm,
maxCumulationBufferCapacity, maxCompositeBufferComponents);
}

@Override
public ChannelPipeline getPipeline() throws Exception {
SSLEngine engine = getSslContext().createSSLEngine();
engine.setUseClientMode(true);

ChannelPipeline pipeline = Channels.pipeline();
pipeline.addLast("ssl", new SslHandler(engine));
SizeHeaderFrameDecoder sizeHeader = new SizeHeaderFrameDecoder();
if (maxCumulationBufferCapacity != null) {
if (maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
sizeHeader.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
sizeHeader.setMaxCumulationBufferCapacity((int) maxCumulationBufferCapacity.bytes());
}
}
if (maxCompositeBufferComponents != -1) {
sizeHeader.setMaxCumulationBufferComponents(maxCompositeBufferComponents);
}
pipeline.addLast("size", sizeHeader);
pipeline.addLast("dispatcher", messageChannelHandler);


return pipeline;
}
}
Loading