Skip to content

Commit

Permalink
Use tls socket to retrieve distinguished name
Browse files Browse the repository at this point in the history
Previously, it was possible to get DN using dn_callback passed to
credentials object. This callback is invoked on every handshake (if
'client_auth' is set to true) and accepts two parameters, subject and
issuer. But there is no way for the user of the tls library to connect
particular client connection with the DN string.

This commit adds a 'get_distinguished_name' method to connected socket
interface. This commit can be used to wait on a future that returns a
distinguished name or nullopt after a handshake. The handshake is
performed asynchrnously. Because of that the retrieval of the DN string
should also be asynchronous.

This commit also adds a unit test and scripts that generate a bunch of
certificates/keys for it.
  • Loading branch information
Lazin committed Apr 26, 2022
1 parent 7109fae commit 3fac4fb
Show file tree
Hide file tree
Showing 11 changed files with 293 additions and 21 deletions.
12 changes: 12 additions & 0 deletions include/seastar/net/api.hh
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,12 @@ struct connected_socket_input_stream_config final {
unsigned max_buffer_size = 128 * 1024;
};

/// Distinguished name
struct session_dn {
sstring subject;
sstring issuer;
};

/// A TCP (or other stream-based protocol) connection.
///
/// A \c connected_socket represents a full-duplex stream between
Expand Down Expand Up @@ -224,6 +230,12 @@ public:
/// This is useful to abort operations on a socket that is not making
/// progress due to a peer failure.
void shutdown_input();

/// Returns DN from client certificate
///
/// The value can only be returned by the server socket and
/// only in case if the client authentication is enabled.
future<std::optional<session_dn>> get_distinguished_name();
};
/// @}

Expand Down
2 changes: 2 additions & 0 deletions include/seastar/net/stack.hh
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
#pragma once

#include <chrono>
#include <optional>
#include <seastar/net/api.hh>
#include <seastar/core/memory.hh>
#include "../core/internal/api-level.hh"
Expand All @@ -46,6 +47,7 @@ public:
virtual keepalive_params get_keepalive_parameters() const = 0;
virtual void set_sockopt(int level, int optname, const void* data, size_t len) = 0;
virtual int get_sockopt(int level, int optname, void* data, size_t len) const = 0;
virtual future<std::optional<session_dn>> get_distinguished_name() = 0;
};

class socket_impl {
Expand Down
6 changes: 3 additions & 3 deletions include/seastar/net/tls.hh
Original file line number Diff line number Diff line change
Expand Up @@ -327,7 +327,7 @@ namespace tls {
/** Wraps an existing connection in SSL/TLS. */
/// @{
future<connected_socket> wrap_client(shared_ptr<certificate_credentials>, connected_socket&&, sstring name = {});
future<connected_socket> wrap_server(shared_ptr<server_credentials>, connected_socket&&);
future<connected_socket> wrap_server(shared_ptr<server_credentials>, connected_socket&&, bool enable_dn_fetch = false);
/// @}

/**
Expand All @@ -337,9 +337,9 @@ namespace tls {
* for the server and optionally trust/crl data.
*/
/// @{
server_socket listen(shared_ptr<server_credentials>, socket_address sa, listen_options opts = listen_options());
server_socket listen(shared_ptr<server_credentials>, socket_address sa, listen_options opts = listen_options(), bool enable_dn_fetch = false);
// Wraps an existing server socket in SSL
server_socket listen(shared_ptr<server_credentials>, server_socket);
server_socket listen(shared_ptr<server_credentials>, server_socket, bool enable_dn_fetch = false);
/// @}
}
}
Expand Down
6 changes: 6 additions & 0 deletions src/net/native-stack-impl.hh
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@ public:
keepalive_params get_keepalive_parameters() const override;
int get_sockopt(int level, int optname, void* data, size_t len) const override;
void set_sockopt(int level, int optname, const void* data, size_t len) override;
future<std::optional<session_dn>> get_distinguished_name() override;
};

template <typename Protocol>
Expand Down Expand Up @@ -264,6 +265,11 @@ int native_connected_socket_impl<Protocol>::get_sockopt(int level, int optname,
throw std::runtime_error("Getting custom socket options is not supported for native stack");
}

template<typename Protocol>
future<std::optional<session_dn>> native_connected_socket_impl<Protocol>::get_distinguished_name() {
return make_ready_future<std::optional<session_dn>>(std::nullopt);
}

}

}
3 changes: 3 additions & 0 deletions src/net/posix-stack.cc
Original file line number Diff line number Diff line change
Expand Up @@ -248,6 +248,9 @@ class posix_connected_socket_impl final : public connected_socket_impl {
int get_sockopt(int level, int optname, void* data, size_t len) const override {
return _ops->get_sockopt(_fd.get_file_desc(), level, optname, data, len);
}
future<std::optional<session_dn>> get_distinguished_name() override {
return make_ready_future<std::optional<session_dn>>(std::nullopt);
}
friend class posix_server_socket_impl;
friend class posix_ap_server_socket_impl;
friend class posix_reuseport_server_socket_impl;
Expand Down
4 changes: 4 additions & 0 deletions src/net/stack.cc
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,10 @@ void connected_socket::shutdown_input() {
_csi->shutdown_input();
}

future<std::optional<session_dn>> connected_socket::get_distinguished_name() {
return _csi->get_distinguished_name();
}

data_source
net::connected_socket_impl::source(connected_socket_input_stream_config csisc) {
// Default implementation falls back to non-parameterized data_source
Expand Down
60 changes: 44 additions & 16 deletions src/net/tls.cc
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
#include <gnutls/gnutls.h>
#include <gnutls/x509.h>
#include <system_error>
#include <variant>

#include <seastar/core/loop.hh>
#include <seastar/core/reactor.hh>
Expand Down Expand Up @@ -972,7 +973,7 @@ class session : public enable_lw_shared_from_this<session> {
};

session(type t, shared_ptr<tls::certificate_credentials> creds,
std::unique_ptr<net::connected_socket_impl> sock, sstring name = { })
std::unique_ptr<net::connected_socket_impl> sock, sstring name = { }, bool enable_dn_fetch = false)
: _type(t), _sock(std::move(sock)), _creds(creds->_impl), _hostname(
std::move(name)), _in(_sock->source()), _out(_sock->sink()),
_in_sem(1), _out_sem(1), _output_pending(
Expand All @@ -995,6 +996,9 @@ class session : public enable_lw_shared_from_this<session> {
gnutls_certificate_server_set_request(*this, GNUTLS_CERT_REQUEST);
break;
case client_auth::REQUIRE:
if (enable_dn_fetch) {
_session_dn = std::make_unique<promise<std::optional<session_dn>>>();
}
gnutls_certificate_server_set_request(*this, GNUTLS_CERT_REQUIRE);
break;
}
Expand All @@ -1018,9 +1022,9 @@ class session : public enable_lw_shared_from_this<session> {
#endif
}
session(type t, shared_ptr<certificate_credentials> creds,
connected_socket sock, sstring name = { })
connected_socket sock, sstring name = { }, bool enable_dn_fetch = false)
: session(t, std::move(creds), net::get_impl::get(std::move(sock)),
std::move(name)) {
std::move(name), enable_dn_fetch) {
}

~session() {
Expand Down Expand Up @@ -1068,6 +1072,7 @@ class session : public enable_lw_shared_from_this<session> {
gnutls_server_name_set(*this, GNUTLS_NAME_DNS, _hostname.data(), _hostname.size());
}
try {
std::optional<session_dn> dn;
auto res = gnutls_handshake(*this);
if (res < 0) {
switch (res) {
Expand Down Expand Up @@ -1184,6 +1189,7 @@ class session : public enable_lw_shared_from_this<session> {
}

void verify() {
bool fetch_dn = static_cast<bool>(_session_dn);
unsigned int status;
auto res = gnutls_certificate_verify_peers3(*this, _type != type::CLIENT || _hostname.empty()
? nullptr : _hostname.c_str(), &status);
Expand All @@ -1198,7 +1204,7 @@ class session : public enable_lw_shared_from_this<session> {
cert_status_to_string(gnutls_certificate_type_get(*this),
status));
}
if (_creds->_dn_callback) {
if (fetch_dn || _creds->_dn_callback) {
// if the user registered a DN (Distinguished Name) callback
// then extract subject and issuer from the (leaf) peer certificate and invoke the callback

Expand All @@ -1217,7 +1223,11 @@ class session : public enable_lw_shared_from_this<session> {
gnutls_x509_crt_deinit(peer_leaf_cert);

if (ec || ec2) {
throw std::runtime_error("error while extracting certificate DN strings");
std::runtime_error err("error while extracting certificate DN strings");
if (fetch_dn) {
_session_dn->set_exception(err);
}
throw err;
}

// a switch here might look overelaborate, however,
Expand All @@ -1232,7 +1242,12 @@ class session : public enable_lw_shared_from_this<session> {
break;
}

_creds->_dn_callback(t, std::move(subject), std::move(issuer));
if (_creds->_dn_callback) {
_creds->_dn_callback(t, subject, issuer);
}
if (fetch_dn) {
_session_dn->set_value(session_dn{.subject=std::move(subject), .issuer=std::move(issuer)});
}
}
}

Expand Down Expand Up @@ -1437,7 +1452,7 @@ class session : public enable_lw_shared_from_this<session> {
if (eof()) {
return make_ready_future<stop_iteration>(stop_iteration::yes);
}
return do_get().then([](auto buf) {
return do_get().then([](auto) {
return make_ready_future<stop_iteration>(stop_iteration::no);
});
});
Expand Down Expand Up @@ -1497,6 +1512,13 @@ class session : public enable_lw_shared_from_this<session> {
return *_sock;
}

future<std::optional<session_dn>> get_distinguished_name() {
if (_session_dn) {
return _session_dn->get_future();
}
return make_ready_future<std::optional<session_dn>>(std::nullopt);
}

struct session_ref;
private:
type _type;
Expand All @@ -1519,6 +1541,7 @@ class session : public enable_lw_shared_from_this<session> {

// modify this to a unique_ptr to handle exceptions in our constructor.
std::unique_ptr<std::remove_pointer_t<gnutls_session_t>, void(*)(gnutls_session_t)> _session;
std::unique_ptr<promise<std::optional<session_dn>>> _session_dn;
};

struct session::session_ref {
Expand Down Expand Up @@ -1586,6 +1609,9 @@ class tls_connected_socket_impl : public net::connected_socket_impl, public sess
int get_sockopt(int level, int optname, void* data, size_t len) const override {
return _session->socket().get_sockopt(level, optname, data, len);
}
future<std::optional<session_dn>> get_distinguished_name() override {
return _session->get_distinguished_name();
}
};


Expand Down Expand Up @@ -1625,15 +1651,15 @@ class tls_connected_socket_impl::sink_impl: public data_sink_impl, public sessio

class server_session : public net::server_socket_impl {
public:
server_session(shared_ptr<server_credentials> creds, server_socket sock)
: _creds(std::move(creds)), _sock(std::move(sock)) {
server_session(shared_ptr<server_credentials> creds, server_socket sock, bool enable_dn_fetch)
: _creds(std::move(creds)), _sock(std::move(sock)), _enable_dn_fetch(enable_dn_fetch) {
}
future<accept_result> accept() override {
// We're not actually doing anything very SSL until we get
// an actual connection. Then we create a "server" session
// and wrap it up after handshaking.
return _sock.accept().then([this](accept_result ar) {
return wrap_server(_creds, std::move(ar.connection)).then([addr = std::move(ar.remote_address)](connected_socket s) {
return wrap_server(_creds, std::move(ar.connection), _enable_dn_fetch).then([addr = std::move(ar.remote_address)](connected_socket s) {
return make_ready_future<accept_result>(accept_result{std::move(s), addr});
});
});
Expand All @@ -1645,8 +1671,10 @@ class server_session : public net::server_socket_impl {
return _sock.local_address();
}
private:

shared_ptr<server_credentials> _creds;
server_socket _sock;
bool _enable_dn_fetch;
};

class tls_socket_impl : public net::socket_impl {
Expand Down Expand Up @@ -1706,18 +1734,18 @@ future<connected_socket> tls::wrap_client(shared_ptr<certificate_credentials> cr
return make_ready_future<connected_socket>(std::move(sock));
}

future<connected_socket> tls::wrap_server(shared_ptr<server_credentials> cred, connected_socket&& s) {
session::session_ref sess(make_lw_shared<session>(session::type::SERVER, std::move(cred), std::move(s)));
future<connected_socket> tls::wrap_server(shared_ptr<server_credentials> cred, connected_socket&& s, bool enable_dn_fetch) {
session::session_ref sess(make_lw_shared<session>(session::type::SERVER, std::move(cred), std::move(s), sstring(), enable_dn_fetch));
connected_socket sock(std::make_unique<tls_connected_socket_impl>(std::move(sess)));
return make_ready_future<connected_socket>(std::move(sock));
}

server_socket tls::listen(shared_ptr<server_credentials> creds, socket_address sa, listen_options opts) {
return listen(std::move(creds), seastar::listen(sa, opts));
server_socket tls::listen(shared_ptr<server_credentials> creds, socket_address sa, listen_options opts, bool enable_dn_fetch) {
return listen(std::move(creds), seastar::listen(sa, opts), enable_dn_fetch);
}

server_socket tls::listen(shared_ptr<server_credentials> creds, server_socket ss) {
server_socket ssls(std::make_unique<server_session>(creds, std::move(ss)));
server_socket tls::listen(shared_ptr<server_credentials> creds, server_socket ss, bool enable_dn_fetch) {
server_socket ssls(std::make_unique<server_session>(creds, std::move(ss), enable_dn_fetch));
return server_socket(std::move(ssls));
}

Expand Down
70 changes: 69 additions & 1 deletion tests/unit/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -528,8 +528,76 @@ function(seastar_add_certgen name)
)
endfunction()

function(seastar_gen_mtls_certs)
find_program(OPENSSL openssl)

set(SUBJECT "/C=GB/ST=London/L=London/O=Redpanda Data/OU=Core/CN=redpanda.com")
set(CLIENT1_SUBJECT "/C=GB/ST=London/L=London/O=Redpanda Data/OU=Core/CN=client1.org")
set(CLIENT2_SUBJECT "/C=GB/ST=London/L=London/O=Redpanda Data/OU=Core/CN=client2.org")

add_custom_command(OUTPUT mtls_ca.key
COMMAND ${OPENSSL} ecparam -name prime256v1 -genkey -noout -out mtls_ca.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
add_custom_command(OUTPUT mtls_ca.crt
COMMAND ${OPENSSL} req -new -x509 -sha256 -key mtls_ca.key -out mtls_ca.crt -subj ${SUBJECT}
DEPENDS mtls_ca.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
# server certificates
add_custom_command(OUTPUT mtls_server.key
COMMAND ${OPENSSL} ecparam -name prime256v1 -genkey -noout -out mtls_server.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
add_custom_command(OUTPUT mtls_server.csr
COMMAND ${OPENSSL} req -new -sha256 -key mtls_server.key -out mtls_server.csr -subj ${SUBJECT}
DEPENDS mtls_server.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
add_custom_command(OUTPUT mtls_server.crt
COMMAND ${OPENSSL} x509 -req -in mtls_server.csr -CA mtls_ca.crt -CAkey mtls_ca.key -CAcreateserial -out mtls_server.crt -days 1000 -sha256
DEPENDS mtls_server.csr mtls_ca.crt mtls_ca.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
# client1 certificates
add_custom_command(OUTPUT mtls_client1.key
COMMAND ${OPENSSL} ecparam -name prime256v1 -genkey -noout -out mtls_client1.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
add_custom_command(OUTPUT mtls_client1.csr
COMMAND ${OPENSSL} req -new -sha256 -key mtls_client1.key -out mtls_client1.csr -subj ${CLIENT1_SUBJECT}
DEPENDS mtls_client1.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
add_custom_command(OUTPUT mtls_client1.crt
COMMAND ${OPENSSL} x509 -req -in mtls_client1.csr -CA mtls_ca.crt -CAkey mtls_ca.key -CAcreateserial -out mtls_client1.crt -days 1000 -sha256
DEPENDS mtls_client1.csr mtls_ca.crt mtls_ca.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
# client2 certificates
add_custom_command(OUTPUT mtls_client2.key
COMMAND ${OPENSSL} ecparam -name prime256v1 -genkey -noout -out mtls_client2.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
add_custom_command(OUTPUT mtls_client2.csr
COMMAND ${OPENSSL} req -new -sha256 -key mtls_client2.key -out mtls_client2.csr -subj ${CLIENT2_SUBJECT}
DEPENDS mtls_client2.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)
add_custom_command(OUTPUT mtls_client2.crt
COMMAND ${OPENSSL} x509 -req -in mtls_client2.csr -CA mtls_ca.crt -CAkey mtls_ca.key -CAcreateserial -out mtls_client2.crt -days 1000 -sha256
DEPENDS mtls_client2.csr mtls_ca.crt mtls_ca.key
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}
)

add_custom_target(mtls_certs
DEPENDS "${CMAKE_CURRENT_BINARY_DIR}/mtls_client1.crt" "${CMAKE_CURRENT_BINARY_DIR}/mtls_client2.crt" "${CMAKE_CURRENT_BINARY_DIR}/mtls_server.crt"
)
endfunction()

seastar_add_certgen(testcrt DOMAIN scylladb.org SERVER test)
seastar_add_certgen(othercrt DOMAIN apa.org SERVER other)
seastar_gen_mtls_certs()

set (tls_certificate_files
tls-ca-bundle.pem
Expand All @@ -555,7 +623,7 @@ add_custom_target(tls_files
)

seastar_add_test (tls
DEPENDS tls_files testcrt othercrt
DEPENDS tls_files testcrt othercrt mtls_certs
SOURCES tls_test.cc
LIBRARIES Boost::filesystem
WORKING_DIRECTORY ${Seastar_BINARY_DIR})
Expand Down
3 changes: 3 additions & 0 deletions tests/unit/loopback_socket.hh
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,9 @@ public:
int get_sockopt(int level, int optname, void* data, size_t len) const override {
throw std::runtime_error("Getting custom socket options is not supported for loopback");
}
future<std::optional<session_dn>> get_distinguished_name() {
throw std::runtime_error("Getting distinguished name is not supported for loopback");
}
};

class loopback_server_socket_impl : public net::server_socket_impl {
Expand Down
Loading

0 comments on commit 3fac4fb

Please sign in to comment.