From 84853dd1fd85dd95c1231d7ff5cecfab8b0dcf5b Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 4 Feb 2020 14:56:40 -0800 Subject: [PATCH 01/44] switched SSL implementation to use boost ssl --- fdbclient/NativeAPI.actor.cpp | 62 ++--- fdbclient/NativeAPI.actor.h | 4 +- fdbrpc/FlowTransport.actor.cpp | 10 +- fdbrpc/TLSConnection.actor.cpp | 6 - fdbrpc/TLSConnection.h | 2 - fdbserver/SimulatedCluster.actor.cpp | 4 - fdbserver/fdbserver.actor.cpp | 37 +-- flow/Net2.actor.cpp | 359 ++++++++++++++++++++++++++- flow/network.h | 3 +- 9 files changed, 409 insertions(+), 78 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 9d9c6dcbed..6452c79eb1 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -66,14 +66,9 @@ using std::max; using std::min; using std::pair; -NetworkOptions networkOptions; -Reference tlsOptions; +#define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) -static void initTLSOptions() { - if (!tlsOptions) { - tlsOptions = Reference(new TLSOptions()); - } -} +NetworkOptions networkOptions; static const Key CLIENT_LATENCY_INFO_PREFIX = LiteralStringRef("client_latency/"); static const Key CLIENT_LATENCY_INFO_CTR_PREFIX = LiteralStringRef("client_latency_counter/"); @@ -887,43 +882,44 @@ void setNetworkOption(FDBNetworkOptions::Option option, Optional valu break; case FDBNetworkOptions::TLS_CERT_PATH: validateOptionValue(value, true); - initTLSOptions(); - tlsOptions->set_cert_file( value.get().toString() ); + networkOptions.sslContext.use_certificate_chain_file(value.get().toString()); break; - case FDBNetworkOptions::TLS_CERT_BYTES: - initTLSOptions(); - tlsOptions->set_cert_data( value.get().toString() ); - break; - case FDBNetworkOptions::TLS_CA_PATH: + case FDBNetworkOptions::TLS_CERT_BYTES: { validateOptionValue(value, true); - initTLSOptions(); - tlsOptions->set_ca_file( value.get().toString() ); + std::string cert = value.get().toString(); + networkOptions.sslContext.use_certificate(boost::asio::buffer(cert.data(), cert.size()), boost::asio::ssl::context::pem); break; - case FDBNetworkOptions::TLS_CA_BYTES: + } + case FDBNetworkOptions::TLS_CA_PATH: { validateOptionValue(value, true); - initTLSOptions(); - tlsOptions->set_ca_data(value.get().toString()); + std::string cert = readFileBytes(value.get().toString(), CERT_FILE_MAX_SIZE); + networkOptions.sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); break; + } + case FDBNetworkOptions::TLS_CA_BYTES: { + validateOptionValue(value, true); + std::string cert = value.get().toString(); + networkOptions.sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); + break; + } case FDBNetworkOptions::TLS_PASSWORD: validateOptionValue(value, true); - initTLSOptions(); - tlsOptions->set_key_password(value.get().toString()); + networkOptions.tlsPassword = value.get().toString(); break; case FDBNetworkOptions::TLS_KEY_PATH: - validateOptionValue(value, true); - initTLSOptions(); - tlsOptions->set_key_file( value.get().toString() ); + validateOptionValue(value, true); + networkOptions.sslContext.use_private_key_file(value.get().toString(), boost::asio::ssl::context::pem); break; - case FDBNetworkOptions::TLS_KEY_BYTES: + case FDBNetworkOptions::TLS_KEY_BYTES: { validateOptionValue(value, true); - initTLSOptions(); - tlsOptions->set_key_data( value.get().toString() ); + std::string cert = value.get().toString(); + networkOptions.sslContext.use_private_key(boost::asio::buffer(cert.data(), cert.size()), boost::asio::ssl::context::pem); break; + } case FDBNetworkOptions::TLS_VERIFY_PEERS: validateOptionValue(value, true); - initTLSOptions(); try { - tlsOptions->set_verify_peers({ value.get().toString() }); + //tlsOptions->set_verify_peers({ value.get().toString() }); FIXME } catch( Error& e ) { TraceEvent(SevWarnAlways, "TLSValidationSetError") .error( e ) @@ -987,15 +983,9 @@ void setupNetwork(uint64_t transportId, bool useMetrics) { if (!networkOptions.logClientInfo.present()) networkOptions.logClientInfo = true; - g_network = newNet2(false, useMetrics || networkOptions.traceDirectory.present()); + g_network = newNet2(false, useMetrics || networkOptions.traceDirectory.present(), &networkOptions.sslContext, networkOptions.tlsPassword); FlowTransport::createInstance(true, transportId); Net2FileSystem::newFileSystem(); - - initTLSOptions(); - -#ifndef TLS_DISABLED - tlsOptions->register_network(); -#endif } void runNetwork() { diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index f02be1b5b3..b480ef2520 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -61,12 +61,14 @@ struct NetworkOptions { Optional logClientInfo; Standalone> supportedVersions; bool slowTaskProfilingEnabled; + boost::asio::ssl::context sslContext; + std::string tlsPassword; // The default values, TRACE_DEFAULT_ROLL_SIZE and TRACE_DEFAULT_MAX_LOGS_SIZE are located in Trace.h. NetworkOptions() : localAddress(""), clusterFile(""), traceDirectory(Optional()), traceRollSize(TRACE_DEFAULT_ROLL_SIZE), traceMaxLogsSize(TRACE_DEFAULT_MAX_LOGS_SIZE), traceLogGroup("default"), - traceFormat("xml"), slowTaskProfilingEnabled(false) {} + traceFormat("xml"), slowTaskProfilingEnabled(false), sslContext(boost::asio::ssl::context(boost::asio::ssl::context::tlsv12)), tlsPassword("") {} }; class Database { diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index ab93d8ae6e..49123c6734 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -988,10 +988,12 @@ ACTOR static Future listen( TransportData* self, NetworkAddress listenAddr try { loop { Reference conn = wait( listener->accept() ); - TraceEvent("ConnectionFrom", conn->getDebugID()).suppressFor(1.0) - .detail("FromAddress", conn->getPeerAddress()) - .detail("ListenAddress", listenAddr.toString()); - incoming.add( connectionIncoming(self, conn) ); + if(conn) { + TraceEvent("ConnectionFrom", conn->getDebugID()).suppressFor(1.0) + .detail("FromAddress", conn->getPeerAddress()) + .detail("ListenAddress", listenAddr.toString()); + incoming.add( connectionIncoming(self, conn) ); + } wait(delay(0) || delay(FLOW_KNOBS->CONNECTION_ACCEPT_DELAY, TaskPriority::WriteSocket)); } } catch (Error& e) { diff --git a/fdbrpc/TLSConnection.actor.cpp b/fdbrpc/TLSConnection.actor.cpp index 7f0f65a1de..e412f9f81c 100644 --- a/fdbrpc/TLSConnection.actor.cpp +++ b/fdbrpc/TLSConnection.actor.cpp @@ -356,12 +356,6 @@ void TLSOptions::set_verify_peers( std::vector const& verify_peers verify_peers_set = true; } -void TLSOptions::register_network() { - // Simulation relies upon being able to call this multiple times, and have it override g_network - // each time it's called. - new TLSNetworkConnections( Reference::addRef( this ) ); -} - ACTOR static Future>> readEntireFile( std::string filename ) { state Reference file = wait(IAsyncFileSystem::filesystem()->open(filename, IAsyncFile::OPEN_READONLY | IAsyncFile::OPEN_UNCACHED, 0)); state int64_t filesize = wait(file->size()); diff --git a/fdbrpc/TLSConnection.h b/fdbrpc/TLSConnection.h index 7e8bb38fd8..af935cc5b9 100644 --- a/fdbrpc/TLSConnection.h +++ b/fdbrpc/TLSConnection.h @@ -85,8 +85,6 @@ struct TLSOptions : ReferenceCounted { void set_key_data( std::string const& key_data ); void set_verify_peers( std::vector const& verify_peers ); - void register_network(); - Reference get_policy(PolicyType type); bool enabled(); diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 65ede31bfd..7a662ef84c 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -117,7 +117,6 @@ static void simInitTLS(Reference tlsOptions) { tlsOptions->set_cert_data( certBytes ); tlsOptions->set_key_data( certBytes ); tlsOptions->set_verify_peers(std::vector(1, "Check.Valid=0")); - tlsOptions->register_network(); } ACTOR Future runBackup( Reference connFile ) { @@ -247,9 +246,6 @@ ACTOR Future simulatedFDBDRebooter(Referenceregister_network(); - } vector> futures; for (int listenPort = port; listenPort < port + listenPerProcess; ++listenPort) { diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index de746b3ac6..6174cd9fa2 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -81,6 +81,8 @@ #include "flow/SimpleOpt.h" #include "flow/actorcompiler.h" // This must be the last #include. +#define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) + enum { OPT_CONNFILE, OPT_SEEDCONNFILE, OPT_SEEDCONNSTRING, OPT_ROLE, OPT_LISTEN, OPT_PUBLICADDR, OPT_DATAFOLDER, OPT_LOGFOLDER, OPT_PARENTPID, OPT_NEWCONSOLE, OPT_NOBOX, OPT_TESTFILE, OPT_RESTARTING, OPT_RESTORING, OPT_RANDOMSEED, OPT_KEY, OPT_MEMLIMIT, OPT_STORAGEMEMLIMIT, OPT_CACHEMEMLIMIT, OPT_MACHINEID, @@ -961,7 +963,7 @@ int main(int argc, char* argv[]) { int minTesterCount = 1; bool testOnServers = false; - Reference tlsOptions = Reference( new TLSOptions ); + boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); std::string tlsCertPath, tlsKeyPath, tlsCAPath, tlsPassword; std::vector tlsVerifyPeers; double fileIoTimeout = 0.0; @@ -1551,7 +1553,21 @@ int main(int argc, char* argv[]) { startNewSimulator(); openTraceFile(NetworkAddress(), rollsize, maxLogsSize, logFolder, "trace", logGroup); } else { - g_network = newNet2(useThreadPool, true); +#ifndef TLS_DISABLED + if ( tlsCertPath.size() ) { + sslContext.use_certificate_chain_file(tlsCertPath); + } + if (tlsCAPath.size()) { + std::string cert = readFileBytes(tlsCAPath, CERT_FILE_MAX_SIZE); + sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); + } + if (tlsKeyPath.size()) { + sslContext.use_private_key_file(tlsKeyPath, boost::asio::ssl::context::pem); + } + //if ( tlsVerifyPeers.size() ) FIXME + // tlsOptions->set_verify_peers( tlsVerifyPeers ); +#endif + g_network = newNet2(useThreadPool, true, &sslContext, tlsPassword); FlowTransport::createInstance(false, 1); const bool expectsPublicAddress = (role == FDBD || role == NetworkTestServer || role == Restore); @@ -1565,22 +1581,7 @@ int main(int argc, char* argv[]) { openTraceFile(publicAddresses.address, rollsize, maxLogsSize, logFolder, "trace", logGroup); -#ifndef TLS_DISABLED - if ( tlsCertPath.size() ) - tlsOptions->set_cert_file( tlsCertPath ); - if (tlsCAPath.size()) - tlsOptions->set_ca_file(tlsCAPath); - if (tlsKeyPath.size()) { - if (tlsPassword.size()) - tlsOptions->set_key_password(tlsPassword); - tlsOptions->set_key_file(tlsKeyPath); - } - if ( tlsVerifyPeers.size() ) - tlsOptions->set_verify_peers( tlsVerifyPeers ); - - tlsOptions->register_network(); -#endif if (expectsPublicAddress) { for (int ii = 0; ii < (publicAddresses.secondaryAddress.present() ? 2 : 1); ++ii) { const NetworkAddress& publicAddress = ii==0 ? publicAddresses.address : publicAddresses.secondaryAddress.get(); @@ -1789,7 +1790,7 @@ int main(int argc, char* argv[]) { } } } - setupAndRun( dataFolder, testFile, restarting, (isRestoring >= 1), whitelistBinPaths, tlsOptions); + setupAndRun( dataFolder, testFile, restarting, (isRestoring >= 1), whitelistBinPaths, Reference()); //FIXME g_simulator.run(); } else if (role == FDBD) { ASSERT( connectionFile ); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 9f3bffb026..659adcf090 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -49,6 +49,7 @@ intptr_t g_stackYieldLimit = 0; using namespace boost::asio::ip; +typedef boost::asio::ssl::stream ssl_socket; #if defined(__linux__) #include @@ -111,7 +112,7 @@ thread_local INetwork* thread_network = 0; class Net2 sealed : public INetwork, public INetworkConnections { public: - Net2(bool useThreadPool, bool useMetrics); + Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, std::string tlsPassword); void run(); void initMetrics(); @@ -154,6 +155,13 @@ public: //private: ASIOReactor reactor; + boost::asio::ssl::context* sslContext; + std::string tlsPassword; + + std::string get_password() const { + return tlsPassword; + } + INetworkConnections *network; // initially this, but can be changed int64_t tsc_begin, tsc_end; @@ -429,6 +437,216 @@ private: } }; +class SSLConnection : public IConnection, ReferenceCounted { +public: + virtual void addref() { ReferenceCounted::addref(); } + virtual void delref() { ReferenceCounted::delref(); } + + virtual void close() { + closeSocket(); + } + + explicit SSLConnection( boost::asio::io_service& io_service, boost::asio::ssl::context& context ) + : id(nondeterministicRandom()->randomUniqueID()), socket(io_service), ssl_sock(socket, context) + { + } + + // This is not part of the IConnection interface, because it is wrapped by INetwork::connect() + ACTOR static Future> connect( boost::asio::io_service* ios, boost::asio::ssl::context* context, NetworkAddress addr ) { + state std::pair peerIP = std::make_pair(addr.ip, addr.port); + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + if (now() < iter->second.second) { + if(iter->second.first >= FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_ATTEMPTS) { + TraceEvent("TLSOutgoingConnectionThrottlingWarning").suppressFor(1.0).detail("PeerIP", addr); + wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT)); + throw connection_failed(); + } + } else { + g_network->networkInfo.serverTLSConnectionThrottler.erase(peerIP); + } + } + + state Reference self( new SSLConnection(*ios, *context) ); + + self->peer_address = addr; + try { + auto to = tcpEndpoint(addr); + BindPromise p("N2_ConnectError", self->id); + Future onConnected = p.getFuture(); + self->socket.async_connect( to, std::move(p) ); + + wait( onConnected ); + try { + BindPromise p("N2_ConnectHandshakeError", self->id); + Future onHandshook = p.getFuture(); + self->ssl_sock.async_handshake( boost::asio::ssl::stream_base::client, std::move(p) ); + wait( onHandshook ); + } catch (Error& e) { + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + iter->second.first++; + } else { + g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_TIMEOUT); + } + throw; + } + + self->init(); + return self; + } catch (Error& e) { + // Either the connection failed, or was cancelled by the caller + self->closeSocket(); + throw; + } + } + + // This is not part of the IConnection interface, because it is wrapped by IListener::accept() + void accept(NetworkAddress peerAddr) { + this->peer_address = peerAddr; + init(); + } + + // returns when write() can write at least one byte + virtual Future onWritable() { + ++g_net2->countWriteProbes; + BindPromise p("N2_WriteProbeError", id); + auto f = p.getFuture(); + socket.async_write_some( boost::asio::null_buffers(), std::move(p) ); + return f; + } + + // returns when read() can read at least one byte + virtual Future onReadable() { + ++g_net2->countReadProbes; + BindPromise p("N2_ReadProbeError", id); + auto f = p.getFuture(); + socket.async_read_some( boost::asio::null_buffers(), std::move(p) ); + return f; + } + + // Reads as many bytes as possible from the read buffer into [begin,end) and returns the number of bytes read (might be 0) + virtual int read( uint8_t* begin, uint8_t* end ) { + boost::system::error_code err; + ++g_net2->countReads; + size_t toRead = end-begin; + size_t size = ssl_sock.read_some( boost::asio::mutable_buffers_1(begin, toRead), err ); + g_net2->bytesReceived += size; + //TraceEvent("ConnRead", this->id).detail("Bytes", size); + if (err) { + if (err == boost::asio::error::would_block) { + ++g_net2->countWouldBlock; + return 0; + } + onReadError(err); + throw connection_failed(); + } + ASSERT( size ); // If the socket is closed, we expect an 'eof' error, not a zero return value + + return size; + } + + // Writes as many bytes as possible from the given SendBuffer chain into the write buffer and returns the number of bytes written (might be 0) + virtual int write( SendBuffer const* data, int limit ) { + boost::system::error_code err; + ++g_net2->countWrites; + + size_t sent = ssl_sock.write_some( boost::iterator_range(SendBufferIterator(data, limit), SendBufferIterator()), err ); + + if (err) { + // Since there was an error, sent's value can't be used to infer that the buffer has data and the limit is positive so check explicitly. + ASSERT(limit > 0); + bool notEmpty = false; + for(auto p = data; p; p = p->next) + if(p->bytes_written - p->bytes_sent > 0) { + notEmpty = true; + break; + } + ASSERT(notEmpty); + + if (err == boost::asio::error::would_block) { + ++g_net2->countWouldBlock; + return 0; + } + onWriteError(err); + throw connection_failed(); + } + + ASSERT( sent ); // Make sure data was sent, and also this check will fail if the buffer chain was empty or the limit was not > 0. + return sent; + } + + virtual NetworkAddress getPeerAddress() { return peer_address; } + + virtual UID getDebugID() { return id; } + + tcp::socket& getSocket() { return socket; } + + ssl_socket& getSSLSocket() { return ssl_sock; } +private: + UID id; + tcp::socket socket; + ssl_socket ssl_sock; + NetworkAddress peer_address; + + struct SendBufferIterator { + typedef boost::asio::const_buffer value_type; + typedef std::forward_iterator_tag iterator_category; + typedef size_t difference_type; + typedef boost::asio::const_buffer* pointer; + typedef boost::asio::const_buffer& reference; + + SendBuffer const* p; + int limit; + + SendBufferIterator(SendBuffer const* p=0, int limit = std::numeric_limits::max()) : p(p), limit(limit) { + ASSERT(limit > 0); + } + + bool operator == (SendBufferIterator const& r) const { return p == r.p; } + bool operator != (SendBufferIterator const& r) const { return p != r.p; } + void operator++() { + limit -= p->bytes_written - p->bytes_sent; + if(limit > 0) + p = p->next; + else + p = NULL; + } + + boost::asio::const_buffer operator*() const { + return boost::asio::const_buffer( p->data + p->bytes_sent, std::min(limit, p->bytes_written - p->bytes_sent) ); + } + }; + + void init() { + // Socket settings that have to be set after connect or accept succeeds + socket.non_blocking(true); + socket.set_option(boost::asio::ip::tcp::no_delay(true)); + platform::setCloseOnExec(socket.native_handle()); + } + + void closeSocket() { + try { + socket.cancel(); + } catch(...) {} + try { + socket.close(); + } catch(...) {} + try { + ssl_sock.shutdown(); + } catch(...) {} + } + + void onReadError( const boost::system::error_code& error ) { + TraceEvent(SevWarn, "N2_ReadError", id).suppressFor(1.0).detail("Message", error.value()); + closeSocket(); + } + void onWriteError( const boost::system::error_code& error ) { + TraceEvent(SevWarn, "N2_WriteError", id).suppressFor(1.0).detail("Message", error.value()); + closeSocket(); + } +}; + class Listener : public IListener, ReferenceCounted { NetworkAddress listenAddress; tcp::acceptor acceptor; @@ -471,6 +689,77 @@ private: } }; +class SSLListener : public IListener, ReferenceCounted { + NetworkAddress listenAddress; + tcp::acceptor acceptor; + boost::asio::ssl::context* context; + +public: + SSLListener( boost::asio::io_service& io_service, boost::asio::ssl::context* context, NetworkAddress listenAddress ) + : listenAddress(listenAddress), acceptor( io_service, tcpEndpoint( listenAddress ) ), context(context) + { + platform::setCloseOnExec(acceptor.native_handle()); + } + + virtual void addref() { ReferenceCounted::addref(); } + virtual void delref() { ReferenceCounted::delref(); } + + // Returns one incoming connection when it is available + virtual Future> accept() { + return doAccept( this ); + } + + virtual NetworkAddress getListenAddress() { return listenAddress; } + +private: + ACTOR static Future> doAccept( SSLListener* self ) { + state Reference conn( new SSLConnection( self->acceptor.get_io_service(), *self->context) ); + state tcp::acceptor::endpoint_type peer_endpoint; + try { + BindPromise p("N2_AcceptError", UID()); + auto f = p.getFuture(); + self->acceptor.async_accept( conn->getSocket(), peer_endpoint, std::move(p) ); + wait( f ); + state IPAddress peer_address = peer_endpoint.address().is_v6() ? IPAddress(peer_endpoint.address().to_v6().to_bytes()) : IPAddress(peer_endpoint.address().to_v4().to_ulong()); + state std::pair peerIP = std::make_pair(peer_address, static_cast(0)); + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + if (now() < iter->second.second) { + if(iter->second.first >= FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_ATTEMPTS) { + TraceEvent("TLSIncomingConnectionThrottlingWarning").suppressFor(1.0).detail("PeerIP", peerIP.first.toString()); + wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT)); + throw connection_failed(); + } + } else { + g_network->networkInfo.serverTLSConnectionThrottler.erase(peerIP); + } + } + + try { + BindPromise p("N2_AcceptHandshakeError", UID()); + auto f = p.getFuture(); + conn->getSSLSocket().async_handshake( boost::asio::ssl::stream_base::server, std::move(p) ); + wait( f ); + } catch (...) { + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + iter->second.first++; + } else { + g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT); + } + throw; + } + + conn->accept(NetworkAddress(peer_address, peer_endpoint.port(), false, true)); + + return conn; + } catch (...) { + conn->close(); + return Reference(); + } + } +}; + struct PromiseTask : public Task, public FastAllocated { Promise promise; PromiseTask() {} @@ -482,7 +771,7 @@ struct PromiseTask : public Task, public FastAllocated { } }; -Net2::Net2(bool useThreadPool, bool useMetrics) +Net2::Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, std::string tlsPassword) : useThreadPool(useThreadPool), network(this), reactor(this), @@ -491,10 +780,16 @@ Net2::Net2(bool useThreadPool, bool useMetrics) // Until run() is called, yield() will always yield tsc_begin(0), tsc_end(0), taskBegin(0), currentTaskID(TaskPriority::DefaultYield), lastMinTaskID(TaskPriority::Zero), - numYields(0) + numYields(0), + sslContext(sslContext), + tlsPassword(tlsPassword) { TraceEvent("Net2Starting"); + if(sslContext) { + sslContext->set_password_callback(std::bind(&Net2::get_password, this)); + } + // Set the global members if(useMetrics) { setGlobal(INetwork::enTDMetrics, (flowGlobalType) &tdmetrics); @@ -870,8 +1165,11 @@ THREAD_HANDLE Net2::startThread( THREAD_FUNC_RETURN (*func) (void*), void *arg ) return ::startThread(func, arg); } - Future< Reference > Net2::connect( NetworkAddress toAddr, std::string host ) { + if ( toAddr.isTLS() ) { + return SSLConnection::connect(&this->reactor.ios, this->sslContext, toAddr); + } + return Connection::connect(&this->reactor.ios, toAddr); } @@ -945,6 +1243,9 @@ bool Net2::isAddressOnThisHost( NetworkAddress const& addr ) { Reference Net2::listen( NetworkAddress localAddr ) { try { + if ( localAddr.isTLS() ) { + return Reference(new SSLListener( reactor.ios, this->sslContext, localAddr )); + } return Reference( new Listener( reactor.ios, localAddr ) ); } catch (boost::system::system_error const& e) { Error x; @@ -1039,9 +1340,55 @@ void ASIOReactor::wake() { } // namespace net2 -INetwork* newNet2(bool useThreadPool, bool useMetrics) { +bool verify_certificate_cb(bool preverified, boost::asio::ssl::verify_context& ctx) +{ + /* + std::cout << "Function : " << __func__ << " ----------------- Line : " << __LINE__ << std::endl; + int8_t subject_name[256]; + X509_STORE_CTX *cts = ctx.native_handle(); + int32_t length = 0; + X509* cert = X509_STORE_CTX_get_current_cert(ctx.native_handle()); + std::cout << "CTX ERROR : " << cts->error << std::endl; + + int32_t depth = X509_STORE_CTX_get_error_depth(cts); + std::cout << "CTX DEPTH : " << depth << std::endl; + + switch (cts->error) + { + case X509_V_ERR_UNABLE_TO_GET_ISSUER_CERT: + printf("X509_V_ERR_UNABLE_TO_GET_ISSUER_CERT\n"); + break; + case X509_V_ERR_CERT_NOT_YET_VALID: + case X509_V_ERR_ERROR_IN_CERT_NOT_BEFORE_FIELD: + printf("Certificate not yet valid!!\n"); + break; + case X509_V_ERR_CERT_HAS_EXPIRED: + case X509_V_ERR_ERROR_IN_CERT_NOT_AFTER_FIELD: + printf("Certificate expired..\n"); + break; + case X509_V_ERR_SELF_SIGNED_CERT_IN_CHAIN: + printf("Self signed certificate in chain!!!\n"); + preverified = true; + break; + default: + break; + } + const int32_t name_length = 256; + X509_NAME_oneline(X509_get_subject_name(cert), reinterpret_cast(subject_name), name_length); + printf("Verifying %s\n", subject_name); + printf("Verification status : %d\n", preverified); + + std::cout << "Function : " << __func__ << " ----------------- Line : " << __LINE__ << std::endl; + */ + return true; +} + +INetwork* newNet2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, std::string tlsPassword) { try { - N2::g_net2 = new N2::Net2(useThreadPool, useMetrics); + sslContext->set_options(boost::asio::ssl::context::default_workarounds); + sslContext->set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); + sslContext->set_verify_callback(boost::bind(&verify_certificate_cb, _1, _2)); + N2::g_net2 = new N2::Net2(useThreadPool, useMetrics, sslContext, tlsPassword); } catch(boost::system::system_error e) { TraceEvent("Net2InitError").detail("Message", e.what()); diff --git a/flow/network.h b/flow/network.h index cecccb5f9a..41a5d384bc 100644 --- a/flow/network.h +++ b/flow/network.h @@ -27,6 +27,7 @@ #include #include #include "boost/asio.hpp" +#include "boost/asio/ssl.hpp" #include "flow/serialize.h" #include "flow/IRandom.h" @@ -390,7 +391,7 @@ typedef NetworkAddressList (*NetworkAddressesFuncPtr)(); class INetwork; extern INetwork* g_network; -extern INetwork* newNet2(bool useThreadPool = false, bool useMetrics = false); +extern INetwork* newNet2(bool useThreadPool = false, bool useMetrics = false, boost::asio::ssl::context* sslContext = nullptr, std::string tlsPassword = ""); class INetwork { public: From c8c34333c18751ebb161685974dab581eb6f2ddd Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 4 Feb 2020 14:59:20 -0800 Subject: [PATCH 02/44] increased connect parallelism --- fdbclient/Knobs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/Knobs.cpp b/fdbclient/Knobs.cpp index 53445bc7c7..f80010f35c 100644 --- a/fdbclient/Knobs.cpp +++ b/fdbclient/Knobs.cpp @@ -202,5 +202,5 @@ ClientKnobs::ClientKnobs(bool randomize) { init( CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME, 7 * 24 * 60 * 60 ); // 7 days //fdbcli - init( CLI_CONNECT_PARALLELISM, 10 ); + init( CLI_CONNECT_PARALLELISM, 200 ); } From c9738ab133c4b8236d617d9c0fe249b2d0915150 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 4 Feb 2020 17:54:03 -0800 Subject: [PATCH 03/44] do not destroy an ssl connection until async_handshake has returned --- flow/Net2.actor.cpp | 62 +++++++++++++++++++++++++++------------------ 1 file changed, 38 insertions(+), 24 deletions(-) diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 659adcf090..b5e8cd397d 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -451,9 +451,43 @@ public: { } + ACTOR static void doConnect( Reference self, Promise connected) { + try { + auto to = tcpEndpoint(self->peer_address); + BindPromise p("N2_ConnectError", self->id); + Future onConnected = p.getFuture(); + self->socket.async_connect( to, std::move(p) ); + + wait( onConnected ); + try { + BindPromise p("N2_ConnectHandshakeError", self->id); + Future onHandshook = p.getFuture(); + self->ssl_sock.async_handshake( boost::asio::ssl::stream_base::client, std::move(p) ); + wait( onHandshook ); + } catch (Error& e) { + std::pair peerIP = std::make_pair(self->peer_address.ip, self->peer_address.port); + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + iter->second.first++; + } else { + g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_TIMEOUT); + } + throw; + } + + self->init(); + connected.send(Void()); + } catch (Error& e) { + // Either the connection failed, or was cancelled by the caller + self->closeSocket(); + connected.sendError(e); + throw; + } + } + // This is not part of the IConnection interface, because it is wrapped by INetwork::connect() ACTOR static Future> connect( boost::asio::io_service* ios, boost::asio::ssl::context* context, NetworkAddress addr ) { - state std::pair peerIP = std::make_pair(addr.ip, addr.port); + std::pair peerIP = std::make_pair(addr.ip, addr.port); auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { if (now() < iter->second.second) { @@ -468,31 +502,11 @@ public: } state Reference self( new SSLConnection(*ios, *context) ); - self->peer_address = addr; + Promise connected; + doConnect(self, connected); try { - auto to = tcpEndpoint(addr); - BindPromise p("N2_ConnectError", self->id); - Future onConnected = p.getFuture(); - self->socket.async_connect( to, std::move(p) ); - - wait( onConnected ); - try { - BindPromise p("N2_ConnectHandshakeError", self->id); - Future onHandshook = p.getFuture(); - self->ssl_sock.async_handshake( boost::asio::ssl::stream_base::client, std::move(p) ); - wait( onHandshook ); - } catch (Error& e) { - auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); - if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { - iter->second.first++; - } else { - g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_TIMEOUT); - } - throw; - } - - self->init(); + wait(connected.getFuture()); return self; } catch (Error& e) { // Either the connection failed, or was cancelled by the caller From 53d0867a17303003614ce5bbfcd2fea4a60b5e83 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 4 Feb 2020 18:15:10 -0800 Subject: [PATCH 04/44] limit the number of connections a process can attempt to establish in parallel --- fdbcli/fdbcli.actor.cpp | 10 +++------- fdbclient/Knobs.cpp | 3 --- fdbclient/Knobs.h | 3 --- fdbrpc/FlowTransport.actor.cpp | 4 ++++ flow/Knobs.cpp | 1 + flow/Knobs.h | 1 + flow/network.cpp | 2 ++ flow/network.h | 3 ++- 8 files changed, 13 insertions(+), 14 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 6ae53ca79d..fd2ac42f45 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -2553,9 +2553,7 @@ Future stopNetworkAfter( Future what ) { } } -ACTOR Future addInterface( std::map>* address_interface, Reference connectLock, KeyValue kv) { - wait(connectLock->take()); - state FlowLock::Releaser releaser(*connectLock); +ACTOR Future addInterface( std::map>* address_interface, KeyValue kv) { state ClientWorkerInterface workerInterf = BinaryReader::fromStringRef(kv.value, IncludeVersion()); state ClientLeaderRegInterface leaderInterf(workerInterf.address()); choose { @@ -2977,10 +2975,9 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { getTransaction(db, tr, options, intrans); if (tokens.size() == 1) { Standalone kvs = wait( makeInterruptable( tr->getRange(KeyRangeRef(LiteralStringRef("\xff\xff/worker_interfaces"), LiteralStringRef("\xff\xff\xff")), 1) ) ); - Reference connectLock(new FlowLock(CLIENT_KNOBS->CLI_CONNECT_PARALLELISM)); std::vector> addInterfs; for( auto it : kvs ) { - addInterfs.push_back(addInterface(&address_interface, connectLock, it)); + addInterfs.push_back(addInterface(&address_interface, it)); } wait( waitForAll(addInterfs) ); } @@ -3291,10 +3288,9 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { getTransaction(db, tr, options, intrans); if (tokens.size() == 1) { Standalone kvs = wait( makeInterruptable( tr->getRange(KeyRangeRef(LiteralStringRef("\xff\xff/worker_interfaces"), LiteralStringRef("\xff\xff\xff")), 1) ) ); - Reference connectLock(new FlowLock(CLIENT_KNOBS->CLI_CONNECT_PARALLELISM)); std::vector> addInterfs; for( auto it : kvs ) { - addInterfs.push_back(addInterface(&address_interface, connectLock, it)); + addInterfs.push_back(addInterface(&address_interface, it)); } wait( waitForAll(addInterfs) ); } diff --git a/fdbclient/Knobs.cpp b/fdbclient/Knobs.cpp index f80010f35c..7b3cba805f 100644 --- a/fdbclient/Knobs.cpp +++ b/fdbclient/Knobs.cpp @@ -200,7 +200,4 @@ ClientKnobs::ClientKnobs(bool randomize) { init( CONSISTENCY_CHECK_RATE_LIMIT_MAX, 50e6 ); // Limit in per sec init( CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME, 7 * 24 * 60 * 60 ); // 7 days - - //fdbcli - init( CLI_CONNECT_PARALLELISM, 200 ); } diff --git a/fdbclient/Knobs.h b/fdbclient/Knobs.h index 25b26b0d12..359e40acde 100644 --- a/fdbclient/Knobs.h +++ b/fdbclient/Knobs.h @@ -191,9 +191,6 @@ public: int CONSISTENCY_CHECK_RATE_LIMIT_MAX; int CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME; - //fdbcli - int CLI_CONNECT_PARALLELISM; - ClientKnobs(bool randomize = false); }; diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 49123c6734..492b3fd486 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -428,10 +428,13 @@ ACTOR Future connectionKeeper( Reference self, self->lastConnectTime = now(); TraceEvent("ConnectingTo", conn ? conn->getDebugID() : UID()).suppressFor(1.0).detail("PeerAddr", self->destination); + wait(g_network->networkInfo.handshakeLock->take()); + state FlowLock::Releaser releaser(*g_network->networkInfo.handshakeLock); try { choose { when( Reference _conn = wait( INetworkConnections::net()->connect(self->destination) ) ) { + releaser.release(); if (FlowTransport::transport().isClient()) { IFailureMonitor::failureMonitor().setStatus(self->destination, FailureStatus(false)); } @@ -453,6 +456,7 @@ ACTOR Future connectionKeeper( Reference self, } } } catch( Error &e ) { + releaser.release(); if(e.code() != error_code_connection_failed) { throw; } diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index fc2e1aa4b6..c6136f8043 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -124,6 +124,7 @@ FlowKnobs::FlowKnobs(bool randomize, bool isSimulated) { init( MAX_PACKET_SEND_BYTES, 256 * 1024 ); init( MIN_PACKET_BUFFER_BYTES, 4 * 1024 ); init( MIN_PACKET_BUFFER_FREE_BYTES, 256 ); + init( HANDSHAKE_LIMIT, 100 ); //Sim2 init( MIN_OPEN_TIME, 0.0002 ); diff --git a/flow/Knobs.h b/flow/Knobs.h index 2904c5dd48..ca4f64d40d 100644 --- a/flow/Knobs.h +++ b/flow/Knobs.h @@ -145,6 +145,7 @@ public: int MAX_PACKET_SEND_BYTES; int MIN_PACKET_BUFFER_BYTES; int MIN_PACKET_BUFFER_FREE_BYTES; + int HANDSHAKE_LIMIT; //Sim2 //FIMXE: more parameters could be factored out diff --git a/flow/network.cpp b/flow/network.cpp index 82c848b5ce..1b7760a45d 100644 --- a/flow/network.cpp +++ b/flow/network.cpp @@ -200,3 +200,5 @@ TEST_CASE("/flow/network/ipaddress") { return Void(); } + +NetworkInfo::NetworkInfo() : handshakeLock( new FlowLock(FLOW_KNOBS->HANDSHAKE_LIMIT) ) {} diff --git a/flow/network.h b/flow/network.h index 41a5d384bc..87564cef96 100644 --- a/flow/network.h +++ b/flow/network.h @@ -328,8 +328,9 @@ struct NetworkInfo { double lastAlternativesFailureSkipDelay = 0; std::map, std::pair> serverTLSConnectionThrottler; + struct FlowLock* handshakeLock; - NetworkInfo() {} + NetworkInfo(); }; class IEventFD : public ReferenceCounted { From 69de4300573ff61d65ee091041332647c0dda2fc Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 6 Feb 2020 16:45:54 -0800 Subject: [PATCH 05/44] separate handshaking from connection to improve pipelining --- fdbrpc/FlowTransport.actor.cpp | 11 +- fdbrpc/TLSConnection.h | 3 + fdbrpc/sim2.actor.cpp | 3 + flow/Knobs.cpp | 2 +- flow/Net2.actor.cpp | 178 ++++++++++++++++++++------------- flow/network.h | 4 + 6 files changed, 127 insertions(+), 74 deletions(-) diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 492b3fd486..36238f93a9 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -428,22 +428,21 @@ ACTOR Future connectionKeeper( Reference self, self->lastConnectTime = now(); TraceEvent("ConnectingTo", conn ? conn->getDebugID() : UID()).suppressFor(1.0).detail("PeerAddr", self->destination); - wait(g_network->networkInfo.handshakeLock->take()); - state FlowLock::Releaser releaser(*g_network->networkInfo.handshakeLock); try { choose { when( Reference _conn = wait( INetworkConnections::net()->connect(self->destination) ) ) { - releaser.release(); + conn = _conn; + wait(conn->connectHandshake()); if (FlowTransport::transport().isClient()) { IFailureMonitor::failureMonitor().setStatus(self->destination, FailureStatus(false)); } if (self->unsent.empty()) { - _conn->close(); + conn->close(); + conn = Reference(); clientReconnectDelay = false; continue; } else { - conn = _conn; TraceEvent("ConnectionExchangingConnectPacket", conn->getDebugID()) .suppressFor(1.0) .detail("PeerAddr", self->destination); @@ -456,7 +455,6 @@ ACTOR Future connectionKeeper( Reference self, } } } catch( Error &e ) { - releaser.release(); if(e.code() != error_code_connection_failed) { throw; } @@ -966,6 +964,7 @@ ACTOR static Future connectionReader( ACTOR static Future connectionIncoming( TransportData* self, Reference conn ) { try { + wait(conn->acceptHandshake()); state Promise> onConnected; state Future reader = connectionReader( self, conn, Reference(), onConnected ); choose { diff --git a/fdbrpc/TLSConnection.h b/fdbrpc/TLSConnection.h index af935cc5b9..b32b6f5f0b 100644 --- a/fdbrpc/TLSConnection.h +++ b/fdbrpc/TLSConnection.h @@ -50,6 +50,9 @@ struct TLSConnection : IConnection, ReferenceCounted { virtual void close() { conn->close(); } + virtual Future acceptHandshake() { return Void(); } + virtual Future connectHandshake() { return Void(); } + virtual Future onWritable(); virtual Future onReadable(); diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 0d5a752091..b944fb833c 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -200,6 +200,9 @@ struct Sim2Conn : IConnection, ReferenceCounted { virtual void delref() { ReferenceCounted::delref(); } virtual void close() { closedByCaller = true; closeInternal(); } + virtual Future acceptHandshake() { return Void(); } + virtual Future connectHandshake() { return Void(); } + virtual Future onWritable() { return whenWritable(this); } virtual Future onReadable() { return whenReadable(this); } diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index c6136f8043..da489ac51b 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -124,7 +124,7 @@ FlowKnobs::FlowKnobs(bool randomize, bool isSimulated) { init( MAX_PACKET_SEND_BYTES, 256 * 1024 ); init( MIN_PACKET_BUFFER_BYTES, 4 * 1024 ); init( MIN_PACKET_BUFFER_FREE_BYTES, 256 ); - init( HANDSHAKE_LIMIT, 100 ); + init( HANDSHAKE_LIMIT, 200 ); //Sim2 init( MIN_OPEN_TIME, 0.0002 ); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index b5e8cd397d..55d80e3e27 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -305,6 +305,10 @@ public: init(); } + virtual Future acceptHandshake() { return Void(); } + + virtual Future connectHandshake() { return Void(); } + // returns when write() can write at least one byte virtual Future onWritable() { ++g_net2->countWriteProbes; @@ -451,40 +455,6 @@ public: { } - ACTOR static void doConnect( Reference self, Promise connected) { - try { - auto to = tcpEndpoint(self->peer_address); - BindPromise p("N2_ConnectError", self->id); - Future onConnected = p.getFuture(); - self->socket.async_connect( to, std::move(p) ); - - wait( onConnected ); - try { - BindPromise p("N2_ConnectHandshakeError", self->id); - Future onHandshook = p.getFuture(); - self->ssl_sock.async_handshake( boost::asio::ssl::stream_base::client, std::move(p) ); - wait( onHandshook ); - } catch (Error& e) { - std::pair peerIP = std::make_pair(self->peer_address.ip, self->peer_address.port); - auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); - if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { - iter->second.first++; - } else { - g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_TIMEOUT); - } - throw; - } - - self->init(); - connected.send(Void()); - } catch (Error& e) { - // Either the connection failed, or was cancelled by the caller - self->closeSocket(); - connected.sendError(e); - throw; - } - } - // This is not part of the IConnection interface, because it is wrapped by INetwork::connect() ACTOR static Future> connect( boost::asio::io_service* ios, boost::asio::ssl::context* context, NetworkAddress addr ) { std::pair peerIP = std::make_pair(addr.ip, addr.port); @@ -503,10 +473,15 @@ public: state Reference self( new SSLConnection(*ios, *context) ); self->peer_address = addr; - Promise connected; - doConnect(self, connected); + try { - wait(connected.getFuture()); + auto to = tcpEndpoint(self->peer_address); + BindPromise p("N2_ConnectError", self->id); + Future onConnected = p.getFuture(); + self->socket.async_connect( to, std::move(p) ); + + wait( onConnected ); + self->init(); return self; } catch (Error& e) { // Either the connection failed, or was cancelled by the caller @@ -521,6 +496,103 @@ public: init(); } + ACTOR static void doAcceptHandshake( Reference self, Promise connected) { + try { + state std::pair peerIP = std::make_pair(self->getPeerAddress().ip, static_cast(0)); + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + if (now() < iter->second.second) { + if(iter->second.first >= FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_ATTEMPTS) { + TraceEvent("TLSIncomingConnectionThrottlingWarning").suppressFor(1.0).detail("PeerIP", peerIP.first.toString()); + wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT)); + throw connection_failed(); + } + } else { + g_network->networkInfo.serverTLSConnectionThrottler.erase(peerIP); + } + } + + wait(g_network->networkInfo.handshakeLock->take()); + state FlowLock::Releaser releaser(*g_network->networkInfo.handshakeLock); + + BindPromise p("N2_AcceptHandshakeError", UID()); + auto onHandshook = p.getFuture(); + self->getSSLSocket().async_handshake( boost::asio::ssl::stream_base::server, std::move(p) ); + wait( onHandshook ); + wait(delay(0) || delay(FLOW_KNOBS->CONNECTION_ACCEPT_DELAY, TaskPriority::WriteSocket)); + connected.send(Void()); + } catch (...) { + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + iter->second.first++; + } else { + g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT); + } + self->closeSocket(); + connected.sendError(connection_failed()); + throw; + } + } + + ACTOR static Future acceptHandshakeWrapper( Reference self ) { + Promise connected; + doAcceptHandshake(self, connected); + try { + wait(connected.getFuture()); + return Void(); + } catch (Error& e) { + // Either the connection failed, or was cancelled by the caller + self->closeSocket(); + throw; + } + } + + virtual Future acceptHandshake() { + return acceptHandshakeWrapper( Reference::addRef(this) ); + } + + ACTOR static void doConnectHandshake( Reference self, Promise connected) { + try { + wait(g_network->networkInfo.handshakeLock->take()); + state FlowLock::Releaser releaser(*g_network->networkInfo.handshakeLock); + + BindPromise p("N2_ConnectHandshakeError", self->id); + Future onHandshook = p.getFuture(); + self->ssl_sock.async_handshake( boost::asio::ssl::stream_base::client, std::move(p) ); + wait( onHandshook ); + wait(delay(0) || delay(FLOW_KNOBS->CONNECTION_ACCEPT_DELAY, TaskPriority::WriteSocket)); + connected.send(Void()); + } catch (...) { + std::pair peerIP = std::make_pair(self->peer_address.ip, self->peer_address.port); + auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); + if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { + iter->second.first++; + } else { + g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_TIMEOUT); + } + self->closeSocket(); + connected.sendError(connection_failed()); + throw; + } + } + + ACTOR static Future connectHandshakeWrapper( Reference self ) { + Promise connected; + doConnectHandshake(self, connected); + try { + wait(connected.getFuture()); + return Void(); + } catch (Error& e) { + // Either the connection failed, or was cancelled by the caller + self->closeSocket(); + throw; + } + } + + virtual Future connectHandshake() { + return connectHandshakeWrapper( Reference::addRef(this) ); + } + // returns when write() can write at least one byte virtual Future onWritable() { ++g_net2->countWriteProbes; @@ -734,42 +806,14 @@ private: auto f = p.getFuture(); self->acceptor.async_accept( conn->getSocket(), peer_endpoint, std::move(p) ); wait( f ); - state IPAddress peer_address = peer_endpoint.address().is_v6() ? IPAddress(peer_endpoint.address().to_v6().to_bytes()) : IPAddress(peer_endpoint.address().to_v4().to_ulong()); - state std::pair peerIP = std::make_pair(peer_address, static_cast(0)); - auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); - if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { - if (now() < iter->second.second) { - if(iter->second.first >= FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_ATTEMPTS) { - TraceEvent("TLSIncomingConnectionThrottlingWarning").suppressFor(1.0).detail("PeerIP", peerIP.first.toString()); - wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT)); - throw connection_failed(); - } - } else { - g_network->networkInfo.serverTLSConnectionThrottler.erase(peerIP); - } - } - - try { - BindPromise p("N2_AcceptHandshakeError", UID()); - auto f = p.getFuture(); - conn->getSSLSocket().async_handshake( boost::asio::ssl::stream_base::server, std::move(p) ); - wait( f ); - } catch (...) { - auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); - if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { - iter->second.first++; - } else { - g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT); - } - throw; - } + auto peer_address = peer_endpoint.address().is_v6() ? IPAddress(peer_endpoint.address().to_v6().to_bytes()) : IPAddress(peer_endpoint.address().to_v4().to_ulong()); conn->accept(NetworkAddress(peer_address, peer_endpoint.port(), false, true)); return conn; } catch (...) { conn->close(); - return Reference(); + throw; } } }; diff --git a/flow/network.h b/flow/network.h index 87564cef96..b86e21660c 100644 --- a/flow/network.h +++ b/flow/network.h @@ -349,6 +349,10 @@ public: // Closes the underlying connection eventually if it is not already closed. virtual void close() = 0; + virtual Future acceptHandshake() = 0; + + virtual Future connectHandshake() = 0; + // returns when write() can write at least one byte (or may throw an error if the connection dies) virtual Future onWritable() = 0; From 38d8d0d6752c3dd825f78389aa576aafccdbdd88 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 6 Feb 2020 19:29:31 -0800 Subject: [PATCH 06/44] fixed simulation --- bindings/flow/fdb_flow.actor.cpp | 3 ++- bindings/flow/tester/Tester.actor.cpp | 6 +++-- fdbclient/NativeAPI.actor.cpp | 2 +- fdbrpc/sim2.actor.cpp | 5 ++-- fdbserver/SimulatedCluster.actor.cpp | 39 +++++++++------------------ fdbserver/SimulatedCluster.h | 2 +- fdbserver/fdbserver.actor.cpp | 4 +-- flow/Net2.actor.cpp | 2 +- flow/network.h | 2 +- 9 files changed, 28 insertions(+), 37 deletions(-) diff --git a/bindings/flow/fdb_flow.actor.cpp b/bindings/flow/fdb_flow.actor.cpp index 810f088e1c..be70d187c5 100644 --- a/bindings/flow/fdb_flow.actor.cpp +++ b/bindings/flow/fdb_flow.actor.cpp @@ -82,7 +82,8 @@ void fdb_flow_test() { fdb->setupNetwork(); startThread(networkThread, fdb); - g_network = newNet2( false ); + boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); + g_network = newNet2( &sslContext, false ); openTraceFile(NetworkAddress(), 1000000, 1000000, "."); systemMonitor(); diff --git a/bindings/flow/tester/Tester.actor.cpp b/bindings/flow/tester/Tester.actor.cpp index 52d193320e..3011aa9987 100644 --- a/bindings/flow/tester/Tester.actor.cpp +++ b/bindings/flow/tester/Tester.actor.cpp @@ -1748,7 +1748,8 @@ ACTOR void startTest(std::string clusterFilename, StringRef prefix, int apiVersi populateOpsThatCreateDirectories(); // FIXME // This is "our" network - g_network = newNet2(false); + boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); + g_network = newNet2(&sslContext, false); ASSERT(!API::isAPIVersionSelected()); try { @@ -1791,7 +1792,8 @@ ACTOR void startTest(std::string clusterFilename, StringRef prefix, int apiVersi ACTOR void _test_versionstamp() { try { - g_network = newNet2(false); + boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); + g_network = newNet2(&sslContext, false); API *fdb = FDB::API::selectAPIVersion(620); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6452c79eb1..ba87f3b8d9 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -983,7 +983,7 @@ void setupNetwork(uint64_t transportId, bool useMetrics) { if (!networkOptions.logClientInfo.present()) networkOptions.logClientInfo = true; - g_network = newNet2(false, useMetrics || networkOptions.traceDirectory.present(), &networkOptions.sslContext, networkOptions.tlsPassword); + g_network = newNet2(&networkOptions.sslContext, false, useMetrics || networkOptions.traceDirectory.present(), networkOptions.tlsPassword); FlowTransport::createInstance(true, transportId); Net2FileSystem::newFileSystem(); } diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index b944fb833c..2578dc9af2 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -1591,10 +1591,10 @@ public: machines.erase(machineId); } - Sim2() : time(0.0), taskCount(0), yielded(false), yield_limit(0), currentTaskID(TaskPriority::Zero) { + Sim2() : time(0.0), taskCount(0), yielded(false), yield_limit(0), currentTaskID(TaskPriority::Zero), sslContext(boost::asio::ssl::context(boost::asio::ssl::context::tlsv12)) { // Not letting currentProcess be NULL eliminates some annoying special cases currentProcess = new ProcessInfo("NoMachine", LocalityData(Optional>(), StringRef(), StringRef(), StringRef()), ProcessClass(), {NetworkAddress()}, this, "", ""); - g_network = net2 = newNet2(false, true); + g_network = net2 = newNet2(&sslContext, false, true); Net2FileSystem::newFileSystem(); check_yield(TaskPriority::Zero); } @@ -1693,6 +1693,7 @@ public: //Sim2Net network; INetwork *net2; + boost::asio::ssl::context sslContext; //Map from machine IP -> machine disk space info std::map diskSpaceMap; diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 7a662ef84c..f57ae5e82b 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -30,7 +30,6 @@ #include "fdbserver/CoordinationInterface.h" #include "fdbmonitor/SimpleIni.h" #include "fdbrpc/AsyncFileNonDurable.actor.h" -#include "fdbrpc/TLSConnection.h" #include "fdbclient/ManagementAPI.actor.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/BackupAgent.actor.h" @@ -113,12 +112,6 @@ T simulate( const T& in ) { return out; } -static void simInitTLS(Reference tlsOptions) { - tlsOptions->set_cert_data( certBytes ); - tlsOptions->set_key_data( certBytes ); - tlsOptions->set_verify_peers(std::vector(1, "Check.Valid=0")); -} - ACTOR Future runBackup( Reference connFile ) { state std::vector> agentFutures; @@ -195,7 +188,7 @@ enum AgentMode { // a loop{} will be needed around the waiting on simulatedFDBD(). For now this simply // takes care of house-keeping such as context switching and file closing. ACTOR Future simulatedFDBDRebooter(Reference connFile, IPAddress ip, - bool sslEnabled, Reference tlsOptions, + bool sslEnabled, uint16_t port, uint16_t listenPerProcess, LocalityData localities, ProcessClass processClass, std::string* dataFolder, std::string* coordFolder, @@ -359,8 +352,7 @@ std::string describe(int const& val) { // Since a datacenter kill is considered to be the same as killing a machine, files cannot be swapped across datacenters std::map< Optional>, std::vector< std::vector< std::string > > > availableFolders; // process count is no longer needed because it is now the length of the vector of ip's, because it was one ip per process -ACTOR Future simulatedMachine(ClusterConnectionString connStr, std::vector ips, bool sslEnabled, - Reference tlsOptions, LocalityData localities, +ACTOR Future simulatedMachine(ClusterConnectionString connStr, std::vector ips, bool sslEnabled, LocalityData localities, ProcessClass processClass, std::string baseFolder, bool restarting, bool useSeedFile, AgentMode runBackupAgents, bool sslOnly, std::string whitelistBinPaths) { state int bootCount = 0; @@ -405,7 +397,7 @@ ACTOR Future simulatedMachine(ClusterConnectionString connStr, std::vector Reference clusterFile(useSeedFile ? new ClusterConnectionFile(path, connStr.toString()) : new ClusterConnectionFile(path)); const int listenPort = i*listenPerProcess + 1; AgentMode agentMode = runBackupAgents == AgentOnly ? ( i == ips.size()-1 ? AgentOnly : AgentNone ) : runBackupAgents; - processes.push_back(simulatedFDBDRebooter(clusterFile, ips[i], sslEnabled, tlsOptions, listenPort, listenPerProcess, localities, processClass, &myFolders[i], &coordFolders[i], baseFolder, connStr, useSeedFile, agentMode, whitelistBinPaths)); + processes.push_back(simulatedFDBDRebooter(clusterFile, ips[i], sslEnabled, listenPort, listenPerProcess, localities, processClass, &myFolders[i], &coordFolders[i], baseFolder, connStr, useSeedFile, agentMode, whitelistBinPaths)); TraceEvent("SimulatedMachineProcess", randomId).detail("Address", NetworkAddress(ips[i], listenPort, true, false)).detail("ZoneId", localities.zoneId()).detail("DataHall", localities.dataHallId()).detail("Folder", myFolders[i]); } @@ -610,7 +602,7 @@ IPAddress makeIPAddressForSim(bool isIPv6, std::array parts) { ACTOR Future restartSimulatedSystem(vector>* systemActors, std::string baseFolder, int* pTesterCount, Optional* pConnString, Standalone* pStartingConfiguration, - Reference tlsOptions, int extraDB, std::string whitelistBinPaths) { + int extraDB, std::string whitelistBinPaths) { CSimpleIni ini; ini.SetUnicode(); ini.LoadFile(joinPath(baseFolder, "restartInfo.ini").c_str()); @@ -706,7 +698,7 @@ ACTOR Future restartSimulatedSystem(vector>* systemActors, st // SOMEDAY: parse backup agent from test file systemActors->push_back(reportErrors( - simulatedMachine(conn, ipAddrs, usingSSL, tlsOptions, localities, processClass, baseFolder, true, + simulatedMachine(conn, ipAddrs, usingSSL, localities, processClass, baseFolder, true, i == useSeedForMachine, enableExtraDB ? AgentAddition : AgentNone, usingSSL && (listenersPerProcess == 1 || processClass == ProcessClass::TesterClass), whitelistBinPaths), processClass == ProcessClass::TesterClass ? "SimulatedTesterMachine" : "SimulatedMachine")); @@ -1083,8 +1075,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR void setupSimulatedSystem(vector>* systemActors, std::string baseFolder, int* pTesterCount, Optional* pConnString, Standalone* pStartingConfiguration, - int extraDB, int minimumReplication, int minimumRegions, Reference tlsOptions, - std::string whitelistBinPaths) { + int extraDB, int minimumReplication, int minimumRegions, std::string whitelistBinPaths) { // SOMEDAY: this does not test multi-interface configurations SimulationConfig simconfig(extraDB, minimumReplication, minimumRegions); StatusObject startingConfigJSON = simconfig.db.toJSON(true); @@ -1155,7 +1146,7 @@ void setupSimulatedSystem(vector>* systemActors, std::string baseFo bool assignClasses = machineCount - dataCenters > 4 && deterministicRandom()->random01() < 0.5; // Use SSL 5% of the time - bool sslEnabled = deterministicRandom()->random01() < 0.10 && tlsOptions->enabled(); + bool sslEnabled = deterministicRandom()->random01() < 0.10; bool sslOnly = sslEnabled && deterministicRandom()->coinflip(); g_simulator.listenersPerProcess = sslEnabled && !sslOnly ? 2 : 1; TEST( sslEnabled ); // SSL enabled @@ -1285,7 +1276,7 @@ void setupSimulatedSystem(vector>* systemActors, std::string baseFo // check the sslEnablementMap using only one ip( LocalityData localities(Optional>(), zoneId, machineId, dcUID); localities.set(LiteralStringRef("data_hall"), dcUID); - systemActors->push_back(reportErrors(simulatedMachine(conn, ips, sslEnabled, tlsOptions, + systemActors->push_back(reportErrors(simulatedMachine(conn, ips, sslEnabled, localities, processClass, baseFolder, false, machine == useSeedForMachine, requiresExtraDBMachines ? AgentOnly : AgentAddition, sslOnly, whitelistBinPaths ), "SimulatedMachine")); if (requiresExtraDBMachines) { @@ -1298,7 +1289,7 @@ void setupSimulatedSystem(vector>* systemActors, std::string baseFo LocalityData localities(Optional>(), newZoneId, newMachineId, dcUID); localities.set(LiteralStringRef("data_hall"), dcUID); - systemActors->push_back(reportErrors(simulatedMachine(*g_simulator.extraDB, extraIps, sslEnabled, tlsOptions, + systemActors->push_back(reportErrors(simulatedMachine(*g_simulator.extraDB, extraIps, sslEnabled, localities, processClass, baseFolder, false, machine == useSeedForMachine, AgentNone, sslOnly, whitelistBinPaths ), "SimulatedMachine")); } @@ -1326,7 +1317,7 @@ void setupSimulatedSystem(vector>* systemActors, std::string baseFo Standalone newZoneId = Standalone(deterministicRandom()->randomUniqueID().toString()); LocalityData localities(Optional>(), newZoneId, newZoneId, Optional>()); systemActors->push_back( reportErrors( simulatedMachine( - conn, ips, sslEnabled, tlsOptions, + conn, ips, sslEnabled, localities, ProcessClass(ProcessClass::TesterClass, ProcessClass::CommandLineSource), baseFolder, false, i == useSeedForMachine, AgentNone, sslEnabled, whitelistBinPaths ), "SimulatedTesterMachine") ); @@ -1384,7 +1375,7 @@ void checkExtraDB(const char *testFile, int &extraDB, int &minimumReplication, i ifs.close(); } -ACTOR void setupAndRun(std::string dataFolder, const char *testFile, bool rebooting, bool restoring, std::string whitelistBinPaths, Reference tlsOptions) { +ACTOR void setupAndRun(std::string dataFolder, const char *testFile, bool rebooting, bool restoring, std::string whitelistBinPaths) { state vector> systemActors; state Optional connFile; state Standalone startingConfiguration; @@ -1405,16 +1396,12 @@ ACTOR void setupAndRun(std::string dataFolder, const char *testFile, bool reboot TaskPriority::DefaultYield)); Sim2FileSystem::newFileSystem(); FlowTransport::createInstance(true, 1); - if (tlsOptions->enabled()) { - simInitTLS(tlsOptions); - } - TEST(true); // Simulation start try { //systemActors.push_back( startSystemMonitor(dataFolder) ); if (rebooting) { - wait( timeoutError( restartSimulatedSystem( &systemActors, dataFolder, &testerCount, &connFile, &startingConfiguration, tlsOptions, extraDB, whitelistBinPaths), 100.0 ) ); + wait( timeoutError( restartSimulatedSystem( &systemActors, dataFolder, &testerCount, &connFile, &startingConfiguration, extraDB, whitelistBinPaths), 100.0 ) ); // FIXME: snapshot restore does not support multi-region restore, hence restore it as single region always if (restoring) { startingConfiguration = LiteralStringRef("usable_regions=1"); @@ -1423,7 +1410,7 @@ ACTOR void setupAndRun(std::string dataFolder, const char *testFile, bool reboot else { g_expect_full_pointermap = 1; setupSimulatedSystem(&systemActors, dataFolder, &testerCount, &connFile, &startingConfiguration, extraDB, - minimumReplication, minimumRegions, tlsOptions, whitelistBinPaths); + minimumReplication, minimumRegions, whitelistBinPaths); wait( delay(1.0) ); // FIXME: WHY!!! //wait for machines to boot } std::string clusterFileDir = joinPath( dataFolder, deterministicRandom()->randomUniqueID().toString() ); diff --git a/fdbserver/SimulatedCluster.h b/fdbserver/SimulatedCluster.h index 85dc44d655..d7ed3ed1ab 100644 --- a/fdbserver/SimulatedCluster.h +++ b/fdbserver/SimulatedCluster.h @@ -24,6 +24,6 @@ #define FDBSERVER_SIMULATEDCLUSTER_H #pragma once -void setupAndRun(std::string const& dataFolder, const char* const& testFile, bool const& rebooting, bool const& restoring, std::string const& whitelistBinPath, Reference const& useSSL); +void setupAndRun(std::string const& dataFolder, const char* const& testFile, bool const& rebooting, bool const& restoring, std::string const& whitelistBinPath); #endif diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index 6174cd9fa2..d3592a207c 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -1567,7 +1567,7 @@ int main(int argc, char* argv[]) { //if ( tlsVerifyPeers.size() ) FIXME // tlsOptions->set_verify_peers( tlsVerifyPeers ); #endif - g_network = newNet2(useThreadPool, true, &sslContext, tlsPassword); + g_network = newNet2(&sslContext, useThreadPool, true, tlsPassword); FlowTransport::createInstance(false, 1); const bool expectsPublicAddress = (role == FDBD || role == NetworkTestServer || role == Restore); @@ -1790,7 +1790,7 @@ int main(int argc, char* argv[]) { } } } - setupAndRun( dataFolder, testFile, restarting, (isRestoring >= 1), whitelistBinPaths, Reference()); //FIXME + setupAndRun( dataFolder, testFile, restarting, (isRestoring >= 1), whitelistBinPaths); g_simulator.run(); } else if (role == FDBD) { ASSERT( connectionFile ); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 55d80e3e27..3df4bed265 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -1441,7 +1441,7 @@ bool verify_certificate_cb(bool preverified, boost::asio::ssl::verify_context& c return true; } -INetwork* newNet2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, std::string tlsPassword) { +INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, bool useMetrics, std::string tlsPassword) { try { sslContext->set_options(boost::asio::ssl::context::default_workarounds); sslContext->set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); diff --git a/flow/network.h b/flow/network.h index b86e21660c..e4140880e8 100644 --- a/flow/network.h +++ b/flow/network.h @@ -396,7 +396,7 @@ typedef NetworkAddressList (*NetworkAddressesFuncPtr)(); class INetwork; extern INetwork* g_network; -extern INetwork* newNet2(bool useThreadPool = false, bool useMetrics = false, boost::asio::ssl::context* sslContext = nullptr, std::string tlsPassword = ""); +extern INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool = false, bool useMetrics = false, std::string tlsPassword = ""); class INetwork { public: From e390dbd36c95a8b17bd4d319aa23771261d4ae8c Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 6 Feb 2020 21:06:52 -0800 Subject: [PATCH 07/44] Add a non-FDBLibTLS verify peers framework to new TLS impl --- fdbclient/NativeAPI.actor.cpp | 18 +- fdbserver/fdbserver.actor.cpp | 9 +- flow/CMakeLists.txt | 2 + flow/Net2.actor.cpp | 52 +--- flow/TLSPolicy.cpp | 493 ++++++++++++++++++++++++++++++++++ flow/TLSPolicy.h | 96 +++++++ flow/flow.vcxproj | 2 + flow/network.h | 3 +- 8 files changed, 623 insertions(+), 52 deletions(-) create mode 100644 flow/TLSPolicy.cpp create mode 100644 flow/TLSPolicy.h diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ba87f3b8d9..f8673cd661 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -44,6 +44,7 @@ #include "flow/Knobs.h" #include "flow/Platform.h" #include "flow/SystemMonitor.h" +#include "flow/TLSPolicy.h" #include "flow/UnitTest.h" #if defined(CMAKE_BUILD) || !defined(WIN32) @@ -69,6 +70,13 @@ using std::pair; #define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) NetworkOptions networkOptions; +static Reference tlsPolicy; + +static void initTLSPolicy() { + if (!tlsPolicy) { + tlsPolicy = Reference(new TLSPolicy(TLSPolicy::Is::CLIENT)); + } +} static const Key CLIENT_LATENCY_INFO_PREFIX = LiteralStringRef("client_latency/"); static const Key CLIENT_LATENCY_INFO_CTR_PREFIX = LiteralStringRef("client_latency_counter/"); @@ -918,11 +926,9 @@ void setNetworkOption(FDBNetworkOptions::Option option, Optional valu } case FDBNetworkOptions::TLS_VERIFY_PEERS: validateOptionValue(value, true); - try { - //tlsOptions->set_verify_peers({ value.get().toString() }); FIXME - } catch( Error& e ) { + initTLSPolicy(); + if (!tlsPolicy->set_verify_peers({ value.get().toString() })) { TraceEvent(SevWarnAlways, "TLSValidationSetError") - .error( e ) .detail("Input", value.get().toString() ); throw invalid_option_value(); } @@ -983,7 +989,9 @@ void setupNetwork(uint64_t transportId, bool useMetrics) { if (!networkOptions.logClientInfo.present()) networkOptions.logClientInfo = true; - g_network = newNet2(&networkOptions.sslContext, false, useMetrics || networkOptions.traceDirectory.present(), networkOptions.tlsPassword); + initTLSPolicy(); + + g_network = newNet2(&networkOptions.sslContext, false, useMetrics || networkOptions.traceDirectory.present(), tlsPolicy, networkOptions.tlsPassword); FlowTransport::createInstance(true, transportId); Net2FileSystem::newFileSystem(); } diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index d3592a207c..f7c25d7d8a 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -58,6 +58,7 @@ #include "fdbrpc/AsyncFileCached.actor.h" #include "fdbserver/CoroFlow.h" #include "flow/SignalSafeUnwind.h" +#include "flow/TLSPolicy.h" #if defined(CMAKE_BUILD) || !defined(WIN32) #include "versions.h" #endif @@ -964,6 +965,7 @@ int main(int argc, char* argv[]) { bool testOnServers = false; boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); + Reference tlsPolicy = Reference(new TLSPolicy(TLSPolicy::Is::SERVER)); std::string tlsCertPath, tlsKeyPath, tlsCAPath, tlsPassword; std::vector tlsVerifyPeers; double fileIoTimeout = 0.0; @@ -1564,10 +1566,11 @@ int main(int argc, char* argv[]) { if (tlsKeyPath.size()) { sslContext.use_private_key_file(tlsKeyPath, boost::asio::ssl::context::pem); } - //if ( tlsVerifyPeers.size() ) FIXME - // tlsOptions->set_verify_peers( tlsVerifyPeers ); + if ( tlsVerifyPeers.size() ) { + tlsPolicy->set_verify_peers( tlsVerifyPeers ); + } #endif - g_network = newNet2(&sslContext, useThreadPool, true, tlsPassword); + g_network = newNet2(&sslContext, useThreadPool, true, tlsPolicy, tlsPassword); FlowTransport::createInstance(false, 1); const bool expectsPublicAddress = (role == FDBD || role == NetworkTestServer || role == Restore); diff --git a/flow/CMakeLists.txt b/flow/CMakeLists.txt index 3075b35ab5..2f9f046bf0 100644 --- a/flow/CMakeLists.txt +++ b/flow/CMakeLists.txt @@ -58,6 +58,8 @@ set(FLOW_SRCS ThreadSafeQueue.h Trace.cpp Trace.h + TLSPolicy.h + TLSPolicy.cpp UnitTest.cpp UnitTest.h XmlTraceLogFormatter.h diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 3df4bed265..7c128944c0 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -37,6 +37,7 @@ #include "flow/AsioReactor.h" #include "flow/Profiler.h" #include "flow/ProtocolVersion.h" +#include "flow/TLSPolicy.h" #ifdef WIN32 #include @@ -1398,54 +1399,19 @@ void ASIOReactor::wake() { } // namespace net2 -bool verify_certificate_cb(bool preverified, boost::asio::ssl::verify_context& ctx) -{ - /* - std::cout << "Function : " << __func__ << " ----------------- Line : " << __LINE__ << std::endl; - int8_t subject_name[256]; - X509_STORE_CTX *cts = ctx.native_handle(); - int32_t length = 0; - X509* cert = X509_STORE_CTX_get_current_cert(ctx.native_handle()); - std::cout << "CTX ERROR : " << cts->error << std::endl; - - int32_t depth = X509_STORE_CTX_get_error_depth(cts); - std::cout << "CTX DEPTH : " << depth << std::endl; - - switch (cts->error) - { - case X509_V_ERR_UNABLE_TO_GET_ISSUER_CERT: - printf("X509_V_ERR_UNABLE_TO_GET_ISSUER_CERT\n"); - break; - case X509_V_ERR_CERT_NOT_YET_VALID: - case X509_V_ERR_ERROR_IN_CERT_NOT_BEFORE_FIELD: - printf("Certificate not yet valid!!\n"); - break; - case X509_V_ERR_CERT_HAS_EXPIRED: - case X509_V_ERR_ERROR_IN_CERT_NOT_AFTER_FIELD: - printf("Certificate expired..\n"); - break; - case X509_V_ERR_SELF_SIGNED_CERT_IN_CHAIN: - printf("Self signed certificate in chain!!!\n"); - preverified = true; - break; - default: - break; - } - const int32_t name_length = 256; - X509_NAME_oneline(X509_get_subject_name(cert), reinterpret_cast(subject_name), name_length); - printf("Verifying %s\n", subject_name); - printf("Verification status : %d\n", preverified); - - std::cout << "Function : " << __func__ << " ----------------- Line : " << __LINE__ << std::endl; - */ - return true; +bool insecurely_always_accept_for_testing(bool _1, boost::asio::ssl::context* _2) { + return true; } -INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, bool useMetrics, std::string tlsPassword) { +INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, bool useMetrics, Reference policy, std::string tlsPassword) { try { sslContext->set_options(boost::asio::ssl::context::default_workarounds); sslContext->set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); - sslContext->set_verify_callback(boost::bind(&verify_certificate_cb, _1, _2)); + if (policy) { + sslContext->set_verify_callback([policy](bool _, boost::asio::ssl::verify_context& ctx) { + return policy->verify_peer(ctx.native_handle()); + }); + } N2::g_net2 = new N2::Net2(useThreadPool, useMetrics, sslContext, tlsPassword); } catch(boost::system::system_error e) { diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp new file mode 100644 index 0000000000..903075abd5 --- /dev/null +++ b/flow/TLSPolicy.cpp @@ -0,0 +1,493 @@ +/* + * TLSPolicy.cpp + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * + * Licensed 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. + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "flow/FastRef.h" +#include "flow/Trace.h" +#include "flow/TLSPolicy.h" + +// To force typeinfo to only be emitted once. +TLSPolicy::~TLSPolicy() {} + +static int hexValue(char c) { + static char const digits[] = "0123456789ABCDEF"; + + if (c >= 'a' && c <= 'f') + c -= ('a' - 'A'); + + int value = std::find(digits, digits + 16, c) - digits; + if (value >= 16) { + throw std::runtime_error("hexValue"); + } + return value; +} + +// Does not handle "raw" form (e.g. #28C4D1), only escaped text +static std::string de4514(std::string const& input, int start, int& out_end) { + std::string output; + + if(input[start] == '#' || input[start] == ' ') { + out_end = start; + return output; + } + + int space_count = 0; + + for(int p = start; p < input.size();) { + switch(input[p]) { + case '\\': // Handle escaped sequence + + // Backslash escaping nothing! + if(p == input.size() - 1) { + out_end = p; + goto FIN; + } + + switch(input[p+1]) { + case ' ': + case '"': + case '#': + case '+': + case ',': + case ';': + case '<': + case '=': + case '>': + case '|': + case '\\': + output += input[p+1]; + p += 2; + space_count = 0; + continue; + + default: + // Backslash escaping pair of hex digits requires two characters + if(p == input.size() - 2) { + out_end = p; + goto FIN; + } + + try { + output += hexValue(input[p+1]) * 16 + hexValue(input[p+2]); + p += 3; + space_count = 0; + continue; + } catch( ... ) { + out_end = p; + goto FIN; + } + } + + case '"': + case '+': + case ',': + case ';': + case '<': + case '>': + case 0: + // All of these must have been escaped + out_end = p; + goto FIN; + + default: + // Character is what it is + output += input[p]; + if(input[p] == ' ') + space_count++; + else + space_count = 0; + p++; + } + } + + out_end = input.size(); + + FIN: + out_end -= space_count; + output.resize(output.size() - space_count); + + return output; +} + +static std::pair splitPair(std::string const& input, char c) { + int p = input.find_first_of(c); + if(p == input.npos) { + throw std::runtime_error("splitPair"); + } + return std::make_pair(input.substr(0, p), input.substr(p+1, input.size())); +} + +static NID abbrevToNID(std::string const& sn) { + NID nid = NID_undef; + + if (sn == "C" || sn == "CN" || sn == "L" || sn == "ST" || sn == "O" || sn == "OU" || sn == "UID" || sn == "DC" || sn == "subjectAltName") + nid = OBJ_sn2nid(sn.c_str()); + if (nid == NID_undef) + throw std::runtime_error("abbrevToNID"); + + return nid; +} + +static X509Location locationForNID(NID nid) { + const char* name = OBJ_nid2ln(nid); + if (name == NULL) { + throw std::runtime_error("locationForNID"); + } + if (strncmp(name, "X509v3", 6) == 0) { + return X509Location::EXTENSION; + } else { + // It probably isn't true that all other NIDs live in the NAME, but it is for now... + return X509Location::NAME; + } +} + +bool TLSPolicy::set_verify_peers(std::vector verify_peers) { + for (int i = 0; i < verify_peers.size(); i++) { + try { + std::string& verifyString = verify_peers[i]; + int start = 0; + while(start < verifyString.size()) { + int split = verifyString.find('|', start); + if(split == std::string::npos) { + break; + } + if(split == start || verifyString[split-1] != '\\') { + rules.emplace_back(verifyString.substr(start,split-start)); + start = split+1; + } + } + rules.emplace_back(verifyString.substr(start)); + } catch ( const std::runtime_error& e ) { + rules.clear(); + std::string& verifyString = verify_peers[i]; + TraceEvent(SevError, "FDBLibTLSVerifyPeersParseError").detail("Config", verifyString); + return false; + } + } + return true; +} + +TLSPolicy::Rule::Rule(std::string input) { + int s = 0; + + while (s < input.size()) { + int eq = input.find('=', s); + + if (eq == input.npos) + throw std::runtime_error("parse_verify"); + + MatchType mt = MatchType::EXACT; + if (input[eq-1] == '>') mt = MatchType::PREFIX; + if (input[eq-1] == '<') mt = MatchType::SUFFIX; + std::string term = input.substr(s, eq - s - (mt == MatchType::EXACT ? 0 : 1)); + + if (term.find("Check.") == 0) { + if (eq + 2 > input.size()) + throw std::runtime_error("parse_verify"); + if (eq + 2 != input.size() && input[eq + 2] != ',') + throw std::runtime_error("parse_verify"); + if (mt != MatchType::EXACT) + throw std::runtime_error("parse_verify: cannot prefix match Check"); + + bool* flag; + + if (term == "Check.Valid") + flag = &verify_cert; + else if (term == "Check.Unexpired") + flag = &verify_time; + else + throw std::runtime_error("parse_verify"); + + if (input[eq + 1] == '0') + *flag = false; + else if (input[eq + 1] == '1') + *flag = true; + else + throw std::runtime_error("parse_verify"); + + s = eq + 3; + } else { + std::map< int, Criteria >* criteria = &subject_criteria; + + if (term.find('.') != term.npos) { + auto scoped = splitPair(term, '.'); + + if (scoped.first == "S" || scoped.first == "Subject") + criteria = &subject_criteria; + else if (scoped.first == "I" || scoped.first == "Issuer") + criteria = &issuer_criteria; + else if (scoped.first == "R" || scoped.first == "Root") + criteria = &root_criteria; + else + throw std::runtime_error("parse_verify"); + + term = scoped.second; + } + + int remain; + auto unesc = de4514(input, eq + 1, remain); + + if (remain == eq + 1) + throw std::runtime_error("parse_verify"); + + NID termNID = abbrevToNID(term); + const X509Location loc = locationForNID(termNID); + criteria->insert(std::make_pair(termNID, Criteria(unesc, mt, loc))); + + if (remain != input.size() && input[remain] != ',') + throw std::runtime_error("parse_verify"); + + s = remain + 1; + } + } +} + +bool match_criteria_entry(const std::string& criteria, ASN1_STRING* entry, MatchType mt) { + bool rc = false; + ASN1_STRING* asn_criteria = NULL; + unsigned char* criteria_utf8 = NULL; + int criteria_utf8_len = 0; + unsigned char* entry_utf8 = NULL; + int entry_utf8_len = 0; + + if ((asn_criteria = ASN1_IA5STRING_new()) == NULL) + goto err; + if (ASN1_STRING_set(asn_criteria, criteria.c_str(), criteria.size()) != 1) + goto err; + if ((criteria_utf8_len = ASN1_STRING_to_UTF8(&criteria_utf8, asn_criteria)) < 1) + goto err; + if ((entry_utf8_len = ASN1_STRING_to_UTF8(&entry_utf8, entry)) < 1) + goto err; + if (mt == MatchType::EXACT) { + if (criteria_utf8_len == entry_utf8_len && + memcmp(criteria_utf8, entry_utf8, criteria_utf8_len) == 0) + rc = true; + } else if (mt == MatchType::PREFIX) { + if (criteria_utf8_len <= entry_utf8_len && + memcmp(criteria_utf8, entry_utf8, criteria_utf8_len) == 0) + rc = true; + } else if (mt == MatchType::SUFFIX) { + if (criteria_utf8_len <= entry_utf8_len && + memcmp(criteria_utf8, entry_utf8 + (entry_utf8_len - criteria_utf8_len), criteria_utf8_len) == 0) + rc = true; + } + + err: + ASN1_STRING_free(asn_criteria); + free(criteria_utf8); + free(entry_utf8); + return rc; +} + +bool match_name_criteria(X509_NAME *name, NID nid, const std::string& criteria, MatchType mt) { + X509_NAME_ENTRY *name_entry; + int idx; + + // If name does not exist, or has multiple of this RDN, refuse to proceed. + if ((idx = X509_NAME_get_index_by_NID(name, nid, -1)) < 0) + return false; + if (X509_NAME_get_index_by_NID(name, nid, idx) != -1) + return false; + if ((name_entry = X509_NAME_get_entry(name, idx)) == NULL) + return false; + + return match_criteria_entry(criteria, X509_NAME_ENTRY_get_data(name_entry), mt); +} + +bool match_extension_criteria(X509 *cert, NID nid, const std::string& value, MatchType mt) { + if (nid != NID_subject_alt_name && nid != NID_issuer_alt_name) { + // I have no idea how other extensions work. + return false; + } + auto pos = value.find(':'); + if (pos == value.npos) { + return false; + } + std::string value_gen = value.substr(0, pos); + std::string value_val = value.substr(pos+1, value.npos); + STACK_OF(GENERAL_NAME)* sans = reinterpret_cast(X509_get_ext_d2i(cert, nid, NULL, NULL)); + if (sans == NULL) { + return false; + } + int num_sans = sk_GENERAL_NAME_num( sans ); + bool rc = false; + for( int i = 0; i < num_sans && !rc; ++i ) { + GENERAL_NAME* altname = sk_GENERAL_NAME_value( sans, i ); + std::string matchable; + switch (altname->type) { + case GEN_OTHERNAME: + break; + case GEN_EMAIL: + if (value_gen == "EMAIL" && + match_criteria_entry( value_val, altname->d.rfc822Name, mt)) { + rc = true; + break; + } + case GEN_DNS: + if (value_gen == "DNS" && + match_criteria_entry( value_val, altname->d.dNSName, mt )) { + rc = true; + break; + } + case GEN_X400: + case GEN_DIRNAME: + case GEN_EDIPARTY: + break; + case GEN_URI: + if (value_gen == "URI" && + match_criteria_entry( value_val, altname->d.uniformResourceIdentifier, mt )) { + rc = true; + break; + } + case GEN_IPADD: + if (value_gen == "IP" && + match_criteria_entry( value_val, altname->d.iPAddress, mt )) { + rc = true; + break; + } + case GEN_RID: + break; + } + } + sk_GENERAL_NAME_pop_free(sans, GENERAL_NAME_free); + return rc; +} + +bool match_criteria(X509* cert, X509_NAME* subject, NID nid, const std::string& criteria, MatchType mt, X509Location loc) { + switch(loc) { + case X509Location::NAME: { + return match_name_criteria(subject, nid, criteria, mt); + } + case X509Location::EXTENSION: { + return match_extension_criteria(cert, nid, criteria, mt); + } + } + // Should never be reachable. + return false; +} + +std::tuple check_verify(const TLSPolicy::Rule* verify, X509_STORE_CTX* store_ctx, bool is_client) { + X509_NAME *subject, *issuer; + bool rc = false; + X509* cert = NULL; + // if returning false, give a reason string + std::string reason = ""; + + // If certificate verification is disabled, there's nothing more to do. + if (!verify->verify_cert) + return std::make_tuple(true, reason); + + //X509_STORE_CTX_trusted_stack(store_ctx, policy->roots); + X509_STORE_CTX_set_default(store_ctx, is_client ? "ssl_server" : "ssl_client"); + if (!verify->verify_time) + X509_VERIFY_PARAM_set_flags(X509_STORE_CTX_get0_param(store_ctx), X509_V_FLAG_NO_CHECK_TIME); + if (X509_verify_cert(store_ctx) <= 0) { + const char *errstr = X509_verify_cert_error_string(X509_STORE_CTX_get_error(store_ctx)); + reason = "Verify cert error: " + std::string(errstr); + goto err; + } + + // Check subject criteria. + cert = sk_X509_value(X509_STORE_CTX_get0_chain(store_ctx), 0); + if ((subject = X509_get_subject_name(cert)) == NULL) { + reason = "Cert subject error"; + goto err; + } + for (auto &pair: verify->subject_criteria) { + if (!match_criteria(cert, subject, pair.first, pair.second.criteria, pair.second.match_type, pair.second.location)) { + reason = "Cert subject match failure"; + goto err; + } + } + + // Check issuer criteria. + if ((issuer = X509_get_issuer_name(cert)) == NULL) { + reason = "Cert issuer error"; + goto err; + } + for (auto &pair: verify->issuer_criteria) { + if (!match_criteria(cert, issuer, pair.first, pair.second.criteria, pair.second.match_type, pair.second.location)) { + reason = "Cert issuer match failure"; + goto err; + } + } + + // Check root criteria - this is the subject of the final certificate in the stack. + cert = sk_X509_value(X509_STORE_CTX_get0_chain(store_ctx), sk_X509_num(X509_STORE_CTX_get0_chain(store_ctx)) - 1); + if ((subject = X509_get_subject_name(cert)) == NULL) { + reason = "Root subject error"; + goto err; + } + for (auto &pair: verify->root_criteria) { + if (!match_criteria(cert, subject, pair.first, pair.second.criteria, pair.second.match_type, pair.second.location)) { + reason = "Root subject match failure"; + goto err; + } + } + + // If we got this far, everything checked out... + rc = true; + + err: + return std::make_tuple(rc, reason); +} + +bool TLSPolicy::verify_peer(X509_STORE_CTX* store_ctx) { + bool rc = false; + std::set verify_failure_reasons; + bool verify_success; + std::string verify_failure_reason; + + TraceEvent("TLSPolicyVerifyPeerCalled"); + + // Any matching rule is sufficient. + for (auto &verify_rule: rules) { + std::tie(verify_success, verify_failure_reason) = check_verify(&verify_rule, store_ctx, is_client); + if (verify_success) { + rc = true; + break; + } else { + if (verify_failure_reason.length() > 0) + verify_failure_reasons.insert(verify_failure_reason); + } + } + + if (!rc) { + // log the various failure reasons + for (std::string reason : verify_failure_reasons) { + TraceEvent("TLSPolicyFailure").suppressFor(1.0).detail("Reason", reason); + } + } + return rc; +} diff --git a/flow/TLSPolicy.h b/flow/TLSPolicy.h new file mode 100644 index 0000000000..a027ae2657 --- /dev/null +++ b/flow/TLSPolicy.h @@ -0,0 +1,96 @@ +/* + * TLSPolicy.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * + * Licensed 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. + */ + +#ifndef _FLOW_TLSPOLICY_H_ +#define _FLOW_TLSPOLICY_H_ +#pragma once + +#include +#include +#include +#include +#include "flow/FastRef.h" + +typedef int NID; + +enum class MatchType { + EXACT, + PREFIX, + SUFFIX, +}; + +enum class X509Location { + // This NID is located within a X509_NAME + NAME, + // This NID is an X509 extension, and should be parsed accordingly + EXTENSION, +}; + +struct Criteria { + Criteria( const std::string& s ) + : criteria(s), match_type(MatchType::EXACT), location(X509Location::NAME) {} + Criteria( const std::string& s, MatchType mt ) + : criteria(s), match_type(mt), location(X509Location::NAME) {} + Criteria( const std::string& s, X509Location loc) + : criteria(s), match_type(MatchType::EXACT), location(loc) {} + Criteria( const std::string& s, MatchType mt, X509Location loc) + : criteria(s), match_type(mt), location(loc) {} + + std::string criteria; + MatchType match_type; + X509Location location; + + bool operator==(const Criteria& c) const { + return criteria == c.criteria && match_type == c.match_type && location == c.location; + } +}; + +class TLSPolicy : ReferenceCounted { +public: + enum class Is { + CLIENT, + SERVER + }; + + bool set_verify_peers(std::vector verify_peers); + bool verify_peer(X509_STORE_CTX* store_ctx); + + TLSPolicy(Is client) : is_client(client == Is::CLIENT) {} + virtual ~TLSPolicy(); + + virtual void addref() { ReferenceCounted::addref(); } + virtual void delref() { ReferenceCounted::delref(); } + + struct Rule { + explicit Rule(std::string input); + + std::map< NID, Criteria > subject_criteria; + std::map< NID, Criteria > issuer_criteria; + std::map< NID, Criteria > root_criteria; + + bool verify_cert = true; + bool verify_time = true; + }; + + std::vector rules; + bool is_client; +}; + +#endif diff --git a/flow/flow.vcxproj b/flow/flow.vcxproj index 401891f223..3a2cc7fd3b 100644 --- a/flow/flow.vcxproj +++ b/flow/flow.vcxproj @@ -49,6 +49,7 @@ + @@ -92,6 +93,7 @@ + diff --git a/flow/network.h b/flow/network.h index e4140880e8..22ff3f4e36 100644 --- a/flow/network.h +++ b/flow/network.h @@ -30,6 +30,7 @@ #include "boost/asio/ssl.hpp" #include "flow/serialize.h" #include "flow/IRandom.h" +#include "flow/TLSPolicy.h" enum class TaskPriority { Max = 1000000, @@ -396,7 +397,7 @@ typedef NetworkAddressList (*NetworkAddressesFuncPtr)(); class INetwork; extern INetwork* g_network; -extern INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool = false, bool useMetrics = false, std::string tlsPassword = ""); +extern INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool = false, bool useMetrics = false, Reference policy = Reference(), std::string tlsPassword = ""); class INetwork { public: From 6b921ac9006261ed85d94241e108a90a381baf5b Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 6 Feb 2020 21:13:58 -0800 Subject: [PATCH 08/44] Stop building FDBLibTLS and stop linking against libtls.so Which now means OpenSSL and LibreSSL are equally acceptable. --- Makefile | 7 ++----- fdbrpc/LoadPlugin.h | 26 ++++++++++---------------- 2 files changed, 12 insertions(+), 21 deletions(-) diff --git a/Makefile b/Makefile index 90d9c0d28f..30d70a81b5 100644 --- a/Makefile +++ b/Makefile @@ -110,8 +110,8 @@ CFLAGS += -DTLS_DISABLED FDB_TLS_LIB := TLS_LIBS := else -FDB_TLS_LIB := lib/libFDBLibTLS.a -TLS_LIBS += $(addprefix $(TLS_LIBDIR)/,libtls.a libssl.a libcrypto.a) +FDB_TLS_LIB := +TLS_LIBS += $(addprefix $(TLS_LIBDIR)/,libssl.a libcrypto.a) endif CXXFLAGS += -Wno-deprecated -DBOOST_ERROR_CODE_HEADER_ONLY -DBOOST_SYSTEM_NO_DEPRECATED @@ -124,9 +124,6 @@ VPATH += $(addprefix :,$(filter-out lib,$(patsubst -L%,%,$(filter -L%,$(LDFLAGS) CS_PROJECTS := flow/actorcompiler flow/coveragetool fdbclient/vexillographer CPP_PROJECTS := flow fdbrpc fdbclient fdbbackup fdbserver fdbcli bindings/c bindings/java fdbmonitor bindings/flow/tester bindings/flow -ifndef TLS_DISABLED -CPP_PROJECTS += FDBLibTLS -endif OTHER_PROJECTS := bindings/python bindings/ruby bindings/go CS_MK_GENERATED := $(CS_PROJECTS:=/generated.mk) diff --git a/fdbrpc/LoadPlugin.h b/fdbrpc/LoadPlugin.h index a74cc4f9b7..9f7024a6f9 100644 --- a/fdbrpc/LoadPlugin.h +++ b/fdbrpc/LoadPlugin.h @@ -18,27 +18,21 @@ * limitations under the License. */ +#ifndef _FLOW_LOADPLUGIN_H_ +#define _FLOW_LOADPLUGIN_H_ + #pragma once -// Specialized TLS plugin library -extern "C" void *get_tls_plugin(const char *plugin_type_name_and_version); - -// Name of specialized TLS Plugin -extern const char* tlsPluginName; +#include +#include "flow/flow.h" template Reference loadPlugin( std::string const& plugin_name ) { void *(*get_plugin)(const char*) = NULL; -#ifndef TLS_DISABLED - if (!plugin_name.compare(tlsPluginName)) { - get_plugin = (void*(*)(const char*)) get_tls_plugin; - } - else -#endif - { - void* plugin = loadLibrary( plugin_name.c_str() ); - if (plugin) - get_plugin = (void*(*)(const char*))loadFunction( plugin, "get_plugin" ); - } + void* plugin = loadLibrary( plugin_name.c_str() ); + if (plugin) + get_plugin = (void*(*)(const char*))loadFunction( plugin, "get_plugin" ); return (get_plugin) ? Reference( (T*)get_plugin( T::get_plugin_type_name_and_version() ) ) : Reference( NULL ); } + +#endif From 2a2bf945ef4bf688513c5bf1aab12366c7526662 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 6 Feb 2020 21:55:13 -0800 Subject: [PATCH 09/44] Also remove FDBLibTLS from CMake --- CMakeLists.txt | 3 --- flow/CMakeLists.txt | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 1826bfdd5d..9ddad782ec 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -174,9 +174,6 @@ set(SEED "0x${SEED_}" CACHE STRING "Random seed for testing") ################################################################################ include(CompileBoost) -if(WITH_TLS) - add_subdirectory(FDBLibTLS) -endif() add_subdirectory(flow) add_subdirectory(fdbrpc) add_subdirectory(fdbclient) diff --git a/flow/CMakeLists.txt b/flow/CMakeLists.txt index 2f9f046bf0..f28379d92b 100644 --- a/flow/CMakeLists.txt +++ b/flow/CMakeLists.txt @@ -99,7 +99,7 @@ endif() if(NOT WITH_TLS) target_compile_definitions(flow PUBLIC TLS_DISABLED) else() - target_link_libraries(flow PUBLIC FDBLibTLS) + target_link_libraries(flow PUBLIC LibreSSL) endif() if(APPLE) From dcbce3593ebe512441021bdb6f5665981ed1bcc1 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Mon, 10 Feb 2020 14:00:21 -0800 Subject: [PATCH 10/44] fixed TLS in simulation --- fdbrpc/sim2.actor.cpp | 13 ++++++------- fdbrpc/simulator.h | 2 +- fdbserver/SimulatedCluster.actor.cpp | 8 ++++---- fdbserver/workloads/ConsistencyCheck.actor.cpp | 5 +++-- flow/network.h | 7 +++++++ 5 files changed, 21 insertions(+), 14 deletions(-) diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 2578dc9af2..6f5d479494 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -804,7 +804,7 @@ public: } // Sets the taskID/priority of the current task, without yielding virtual Future> connect( NetworkAddress toAddr, std::string host ) { - ASSERT( !toAddr.isTLS() && host.empty()); + ASSERT( host.empty()); if (!addressMap.count( toAddr )) { return waitForProcessAndConnect( toAddr, this ); } @@ -822,7 +822,7 @@ public: } else { localIp = IPAddress(getCurrentProcess()->address.ip.toV4() + deterministicRandom()->randomInt(0, 256)); } - peerc->connect(myc, NetworkAddress(localIp, deterministicRandom()->randomInt(40000, 60000))); + peerc->connect(myc, NetworkAddress(localIp, deterministicRandom()->randomInt(40000, 60000), false, toAddr.isTLS())); ((Sim2Listener*)peerp->getListener(toAddr).getPtr())->incomingConnection( 0.5*deterministicRandom()->random01(), Reference(peerc) ); return onConnect( ::delay(0.5*deterministicRandom()->random01()), myc ); @@ -843,7 +843,6 @@ public: return conn; } virtual Reference listen( NetworkAddress localAddr ) { - ASSERT( !localAddr.isTLS() ); Reference listener( getCurrentProcess()->getListener(localAddr) ); ASSERT(listener); return listener; @@ -998,7 +997,7 @@ public: virtual void run() { _run(this); } - virtual ProcessInfo* newProcess(const char* name, IPAddress ip, uint16_t port, uint16_t listenPerProcess, + virtual ProcessInfo* newProcess(const char* name, IPAddress ip, uint16_t port, bool sslEnabled, uint16_t listenPerProcess, LocalityData locality, ProcessClass startingClass, const char* dataFolder, const char* coordinationFolder) { ASSERT( locality.machineId().present() ); @@ -1027,14 +1026,14 @@ public: } NetworkAddressList addresses; - addresses.address = NetworkAddress(ip, port, true, false); + addresses.address = NetworkAddress(ip, port, true, sslEnabled); if(listenPerProcess == 2) { addresses.secondaryAddress = NetworkAddress(ip, port+1, true, false); } ProcessInfo* m = new ProcessInfo(name, locality, startingClass, addresses, this, dataFolder, coordinationFolder); for (int processPort = port; processPort < port + listenPerProcess; ++processPort) { - NetworkAddress address(ip, processPort, true, false); // SOMEDAY see above about becoming SSL! + NetworkAddress address(ip, processPort, true, sslEnabled && processPort == port); m->listenerMap[address] = Reference( new Sim2Listener(m, address) ); addressMap[address] = m; } @@ -1567,7 +1566,7 @@ public: return processes; } virtual ProcessInfo* getProcessByAddress( NetworkAddress const& address ) { - NetworkAddress normalizedAddress(address.ip, address.port, true, false); + NetworkAddress normalizedAddress(address.ip, address.port, true, address.isTLS()); ASSERT( addressMap.count( normalizedAddress ) ); return addressMap[ normalizedAddress ]; } diff --git a/fdbrpc/simulator.h b/fdbrpc/simulator.h index c78fd82edb..9d0a153a75 100644 --- a/fdbrpc/simulator.h +++ b/fdbrpc/simulator.h @@ -140,7 +140,7 @@ public: virtual Future onProcess( ISimulator::ProcessInfo *process, TaskPriority taskID = TaskPriority::Zero ) = 0; virtual Future onMachine( ISimulator::ProcessInfo *process, TaskPriority taskID = TaskPriority::Zero ) = 0; - virtual ProcessInfo* newProcess(const char* name, IPAddress ip, uint16_t port, uint16_t listenPerProcess, + virtual ProcessInfo* newProcess(const char* name, IPAddress ip, uint16_t port, bool sslEnabled, uint16_t listenPerProcess, LocalityData locality, ProcessClass startingClass, const char* dataFolder, const char* coordinationFolder) = 0; virtual void killProcess( ProcessInfo* machine, KillType ) = 0; diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index f57ae5e82b..81a01eb520 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -210,7 +210,7 @@ ACTOR Future simulatedFDBDRebooter(Referencec_str(), + g_simulator.newProcess("Server", ip, port, sslEnabled, listenPerProcess, localities, processClass, dataFolder->c_str(), coordFolder->c_str()); wait(g_simulator.onProcess(process, TaskPriority::DefaultYield)); // Now switch execution to the process on which we will run @@ -1317,9 +1317,9 @@ void setupSimulatedSystem(vector>* systemActors, std::string baseFo Standalone newZoneId = Standalone(deterministicRandom()->randomUniqueID().toString()); LocalityData localities(Optional>(), newZoneId, newZoneId, Optional>()); systemActors->push_back( reportErrors( simulatedMachine( - conn, ips, sslEnabled, + conn, ips, sslEnabled && sslOnly, localities, ProcessClass(ProcessClass::TesterClass, ProcessClass::CommandLineSource), - baseFolder, false, i == useSeedForMachine, AgentNone, sslEnabled, whitelistBinPaths ), + baseFolder, false, i == useSeedForMachine, AgentNone, sslEnabled && sslOnly, whitelistBinPaths ), "SimulatedTesterMachine") ); } *pStartingConfiguration = startingConfigString; @@ -1387,7 +1387,7 @@ ACTOR void setupAndRun(std::string dataFolder, const char *testFile, bool reboot // TODO (IPv6) Use IPv6? wait(g_simulator.onProcess( - g_simulator.newProcess("TestSystem", IPAddress(0x01010101), 1, 1, + g_simulator.newProcess("TestSystem", IPAddress(0x01010101), 1, false, 1, LocalityData(Optional>(), Standalone(deterministicRandom()->randomUniqueID().toString()), Standalone(deterministicRandom()->randomUniqueID().toString()), diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 9c6760e78b..2dc42611e9 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -1221,12 +1221,13 @@ struct ConsistencyCheckWorkload : TestWorkload std::set workerAddresses; for (const auto& it : workers) { - ISimulator::ProcessInfo* info = g_simulator.getProcessByAddress(it.interf.address()); + NetworkAddress addr = it.interf.tLog.getEndpoint().addresses.getTLSAddress(); + ISimulator::ProcessInfo* info = g_simulator.getProcessByAddress(addr); if(!info || info->failed) { TraceEvent("ConsistencyCheck_FailedWorkerInList").detail("Addr", it.interf.address()); return false; } - workerAddresses.insert( NetworkAddress(it.interf.address().ip, it.interf.address().port, true, false) ); + workerAddresses.insert( NetworkAddress(addr.ip, addr.port, true, addr.isTLS()) ); } vector all = g_simulator.getAllProcesses(); diff --git a/flow/network.h b/flow/network.h index 22ff3f4e36..c7d292b747 100644 --- a/flow/network.h +++ b/flow/network.h @@ -284,6 +284,13 @@ struct NetworkAddressList { return secondaryAddress < r.secondaryAddress; } + NetworkAddress getTLSAddress() const { + if(!secondaryAddress.present() || address.isTLS()) { + return address; + } + return secondaryAddress.get(); + } + std::string toString() const { if(!secondaryAddress.present()) { return address.toString(); From 38a5511b9631901d4281620519273f902cf2ac14 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Tue, 11 Feb 2020 15:52:06 -0800 Subject: [PATCH 11/44] additional simulation fixes --- fdbclient/ManagementAPI.actor.cpp | 2 +- fdbserver/SimulatedCluster.actor.cpp | 2 +- fdbserver/workloads/ConsistencyCheck.actor.cpp | 7 ++++--- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 14cd25abe5..5cc4f5b8c9 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -955,7 +955,7 @@ ACTOR Future changeQuorum( Database cx, ReferenceisSimulated()) { for(int i = 0; i < (desiredCoordinators.size()/2)+1; i++) { - auto address = NetworkAddress(desiredCoordinators[i].ip,desiredCoordinators[i].port,true,false); + auto address = NetworkAddress(desiredCoordinators[i].ip,desiredCoordinators[i].port,true,desiredCoordinators[i].isTLS()); g_simulator.protectedAddresses.insert(address); TraceEvent("ProtectCoordinator").detail("Address", address).backtrace(); } diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 81a01eb520..317af9cc7a 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -1205,7 +1205,7 @@ void setupSimulatedSystem(vector>* systemActors, std::string baseFo .detail("Address", coordinatorAddresses[i]) .detail("Coordinators", describe(coordinatorAddresses)); g_simulator.protectedAddresses.insert( - NetworkAddress(coordinatorAddresses[i].ip, coordinatorAddresses[i].port, true, false)); + NetworkAddress(coordinatorAddresses[i].ip, coordinatorAddresses[i].port, true, coordinatorAddresses[i].isTLS())); if(coordinatorAddresses[i].port==2) { g_simulator.protectedAddresses.insert(NetworkAddress(coordinatorAddresses[i].ip, 1, true, false)); } diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index 2dc42611e9..184c1ac0ed 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -1137,18 +1137,19 @@ struct ConsistencyCheckWorkload : TestWorkload std::set> missingStorage; for( int i = 0; i < workers.size(); i++ ) { - if( !configuration.isExcludedServer(workers[i].interf.address()) && + NetworkAddress addr = workers[i].interf.tLog.getEndpoint().addresses.getTLSAddress(); + if( !configuration.isExcludedServer(addr) && ( workers[i].processClass == ProcessClass::StorageClass || workers[i].processClass == ProcessClass::UnsetClass ) ) { bool found = false; for( int j = 0; j < storageServers.size(); j++ ) { - if( storageServers[j].address() == workers[i].interf.address() ) { + if( storageServers[j].getVersion.getEndpoint().addresses.getTLSAddress() == addr ) { found = true; break; } } if( !found ) { TraceEvent("ConsistencyCheck_NoStorage") - .detail("Address", workers[i].interf.address()) + .detail("Address", addr) .detail("ProcessClassEqualToStorageClass", (int)(workers[i].processClass == ProcessClass::StorageClass)); missingStorage.insert(workers[i].interf.locality.dcId()); From 96eec756b3705c865be6704d524779c0ed96aeee Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 12 Feb 2020 15:12:43 -0800 Subject: [PATCH 12/44] more simulation fixes --- fdbclient/ManagementAPI.actor.cpp | 10 +++++++--- fdbserver/SimulatedCluster.actor.cpp | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 5cc4f5b8c9..6311bb0c06 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -955,9 +955,13 @@ ACTOR Future changeQuorum( Database cx, ReferenceisSimulated()) { for(int i = 0; i < (desiredCoordinators.size()/2)+1; i++) { - auto address = NetworkAddress(desiredCoordinators[i].ip,desiredCoordinators[i].port,true,desiredCoordinators[i].isTLS()); - g_simulator.protectedAddresses.insert(address); - TraceEvent("ProtectCoordinator").detail("Address", address).backtrace(); + auto addresses = g_simulator.getProcessByAddress(desiredCoordinators[i])->addresses; + + g_simulator.protectedAddresses.insert(addresses.address); + if(addresses.secondaryAddress.present()) { + g_simulator.protectedAddresses.insert(addresses.secondaryAddress.get()); + } + TraceEvent("ProtectCoordinator").detail("Address", desiredCoordinators[i]).backtrace(); } } diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 317af9cc7a..7412f98b14 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -1207,7 +1207,7 @@ void setupSimulatedSystem(vector>* systemActors, std::string baseFo g_simulator.protectedAddresses.insert( NetworkAddress(coordinatorAddresses[i].ip, coordinatorAddresses[i].port, true, coordinatorAddresses[i].isTLS())); if(coordinatorAddresses[i].port==2) { - g_simulator.protectedAddresses.insert(NetworkAddress(coordinatorAddresses[i].ip, 1, true, false)); + g_simulator.protectedAddresses.insert(NetworkAddress(coordinatorAddresses[i].ip, 1, true, true)); } } deterministicRandom()->randomShuffle(coordinatorAddresses); From 8298fb3cb5d8d4dfbeb20999e93bc28ac8e4b45a Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Wed, 12 Feb 2020 14:39:40 -0800 Subject: [PATCH 13/44] Remove spammy traceevent from testing --- flow/TLSPolicy.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp index 903075abd5..1b6b3a540b 100644 --- a/flow/TLSPolicy.cpp +++ b/flow/TLSPolicy.cpp @@ -469,8 +469,6 @@ bool TLSPolicy::verify_peer(X509_STORE_CTX* store_ctx) { bool verify_success; std::string verify_failure_reason; - TraceEvent("TLSPolicyVerifyPeerCalled"); - // Any matching rule is sufficient. for (auto &verify_rule: rules) { std::tie(verify_success, verify_failure_reason) = check_verify(&verify_rule, store_ctx, is_client); From f2d30a99540d3a04ad979080a1fa54a050867052 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Wed, 12 Feb 2020 14:40:17 -0800 Subject: [PATCH 14/44] comment out certBytes to fix cmake builds --- fdbserver/SimulatedCluster.actor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 7412f98b14..144bbce538 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -48,6 +48,8 @@ const int MACHINE_REBOOT_TIME = 10; bool destructed = false; +/* +FIXME: Remove this? static const char* certBytes = "-----BEGIN CERTIFICATE-----\n" "MIIEGzCCAwOgAwIBAgIJANUQj1rRA2XMMA0GCSqGSIb3DQEBBQUAMIGjMQswCQYD\n" @@ -101,6 +103,7 @@ static const char* certBytes = "iastGId8HyONy3UPGPxCn4b95cIxKvdpt+hvWtYHIBCfHXluQK7zsDMgvtXjYNiz\n" "peZRikYlwmu1K2YRTf7oLE2Ogw==\n" "-----END PRIVATE KEY-----\n"; +*/ template T simulate( const T& in ) { From d716c50000f40664b2f3ab5b6608e12970dae184 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 13 Feb 2020 14:42:45 -0800 Subject: [PATCH 15/44] Find OpenSSL or LibreSSL in CMake --- cmake/FDBComponents.cmake | 17 +++++++++++------ flow/CMakeLists.txt | 2 +- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/cmake/FDBComponents.cmake b/cmake/FDBComponents.cmake index 69e93cec06..b425b495f2 100644 --- a/cmake/FDBComponents.cmake +++ b/cmake/FDBComponents.cmake @@ -9,21 +9,26 @@ if(USE_VALGRIND) endif() ################################################################################ -# LibreSSL +# SSL ################################################################################ set(DISABLE_TLS OFF CACHE BOOL "Don't try to find LibreSSL and always build without TLS support") if(DISABLE_TLS) set(WITH_TLS OFF) else() - set(LIBRESSL_USE_STATIC_LIBS TRUE) - find_package(LibreSSL) - if(LibreSSL_FOUND) + set(OPENSSL_USE_STATIC_LIBS TRUE) + find_package(OpenSSL) + if(NOT OPENSSL_FOUND) + set(LIBRESSL_USE_STATIC_LIBS TRUE) + find_package(LibreSSL) + add_library(OpenSSL::SSL ALIAS LibreSSL) + endif() + if(OPENSSL_FOUND OR LIBRESSL_FOUND) set(WITH_TLS ON) add_compile_options(-DHAVE_OPENSSL) else() - message(STATUS "LibreSSL NOT Found - Will compile without TLS Support") - message(STATUS "You can set LibreSSL_ROOT to the LibreSSL install directory to help cmake find it") + message(STATUS "Neither OpenSSL nor LibreSSL were found - Will compile without TLS Support") + message(STATUS "You can set OPENSSL_ROOT_DIR or LibreSSL_ROOT to the LibreSSL install directory to help cmake find it") set(WITH_TLS OFF) endif() endif() diff --git a/flow/CMakeLists.txt b/flow/CMakeLists.txt index f28379d92b..dbccb2f621 100644 --- a/flow/CMakeLists.txt +++ b/flow/CMakeLists.txt @@ -99,7 +99,7 @@ endif() if(NOT WITH_TLS) target_compile_definitions(flow PUBLIC TLS_DISABLED) else() - target_link_libraries(flow PUBLIC LibreSSL) + target_link_libraries(flow PUBLIC OpenSSL::SSL) endif() if(APPLE) From 723a70b357045e097b68fdd4eee76aa7ca9917aa Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 13 Feb 2020 21:30:57 -0800 Subject: [PATCH 16/44] Call X509_verify_cert once and implement time checking by hand --- flow/TLSPolicy.cpp | 45 ++++++++++++++++++++++++++++++++++----------- 1 file changed, 34 insertions(+), 11 deletions(-) diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp index 1b6b3a540b..2074e6d759 100644 --- a/flow/TLSPolicy.cpp +++ b/flow/TLSPolicy.cpp @@ -404,18 +404,25 @@ std::tuple check_verify(const TLSPolicy::Rule* verify, X509_ST // if returning false, give a reason string std::string reason = ""; - // If certificate verification is disabled, there's nothing more to do. - if (!verify->verify_cert) - return std::make_tuple(true, reason); + if (verify->verify_time) { + X509_VERIFY_PARAM* param = X509_STORE_CTX_get0_param(store_ctx); + time_t cert_time = X509_VERIFY_PARAM_get_time(param); + STACK_OF(X509) *sk = X509_STORE_CTX_get0_chain(store_ctx); + for (int i = 0; i < sk_X509_num(sk); i++) { + X509* cert = sk_X509_value(sk, i); - //X509_STORE_CTX_trusted_stack(store_ctx, policy->roots); - X509_STORE_CTX_set_default(store_ctx, is_client ? "ssl_server" : "ssl_client"); - if (!verify->verify_time) - X509_VERIFY_PARAM_set_flags(X509_STORE_CTX_get0_param(store_ctx), X509_V_FLAG_NO_CHECK_TIME); - if (X509_verify_cert(store_ctx) <= 0) { - const char *errstr = X509_verify_cert_error_string(X509_STORE_CTX_get_error(store_ctx)); - reason = "Verify cert error: " + std::string(errstr); - goto err; + int rv = X509_cmp_time(X509_get0_notBefore(cert), &cert_time); + if (rv >= 0) { + reason = "Cert is not yet valid"; + goto err; + } + + rv = X509_cmp_time(X509_get0_notAfter(cert), &cert_time); + if (rv <= 0) { + reason = "Cert has expired"; + goto err; + } + } } // Check subject criteria. @@ -464,11 +471,27 @@ std::tuple check_verify(const TLSPolicy::Rule* verify, X509_ST } bool TLSPolicy::verify_peer(X509_STORE_CTX* store_ctx) { + //??? + //X509_STORE_CTX* store_ctx = X509_STORE_CTX_new(); + //X509_STORE_CTX_init(store_ctx, X509_STORE_CTX_get0_store(store_ctx_), X509_STORE_CTX_get0_cert(store_ctx_), X509_STORE_CTX_get0_chain(store_ctx_)); bool rc = false; std::set verify_failure_reasons; bool verify_success; std::string verify_failure_reason; + // If certificate verification is disabled, there's nothing more to do. + if (std::any_of(rules.begin(), rules.end(), [](const Rule& r){ return r.verify_cert; })) + return true; + + X509_STORE_CTX_set_default(store_ctx, is_client ? "ssl_server" : "ssl_client"); + X509_VERIFY_PARAM_set_flags(X509_STORE_CTX_get0_param(store_ctx), X509_V_FLAG_NO_CHECK_TIME); + if (X509_verify_cert(store_ctx) <= 0) { + const char *errstr = X509_verify_cert_error_string(X509_STORE_CTX_get_error(store_ctx)); + verify_failure_reason = "Verify cert error: " + std::string(errstr); + TraceEvent("TLSPolicyFailure").suppressFor(1.0).detail("Reason", verify_failure_reason); + return false; + } + // Any matching rule is sufficient. for (auto &verify_rule: rules) { std::tie(verify_success, verify_failure_reason) = check_verify(&verify_rule, store_ctx, is_client); From c859f859bca90573ce0856ace28613bb005b5ecc Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 13 Feb 2020 21:34:23 -0800 Subject: [PATCH 17/44] Remove certBytes. --- fdbserver/SimulatedCluster.actor.cpp | 57 ---------------------------- 1 file changed, 57 deletions(-) diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 144bbce538..f28cdab6e6 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -48,63 +48,6 @@ const int MACHINE_REBOOT_TIME = 10; bool destructed = false; -/* -FIXME: Remove this? -static const char* certBytes = - "-----BEGIN CERTIFICATE-----\n" - "MIIEGzCCAwOgAwIBAgIJANUQj1rRA2XMMA0GCSqGSIb3DQEBBQUAMIGjMQswCQYD\n" - "VQQGEwJVUzELMAkGA1UECAwCVkExDzANBgNVBAcMBlZpZW5uYTEaMBgGA1UECgwR\n" - "Rm91bmRhdGlvbkRCLCBMTEMxGTAXBgNVBAsMEFRlc3QgZW5naW5lZXJpbmcxFTAT\n" - "BgNVBAMMDE1yLiBCaWcgVHVuYTEoMCYGCSqGSIb3DQEJARYZYmlnLnR1bmFAZm91\n" - "bmRhdGlvbmRiLmNvbTAeFw0xNDEyMDUxNTEyMjFaFw0yNDEyMDIxNTEyMjFaMIGj\n" - "MQswCQYDVQQGEwJVUzELMAkGA1UECAwCVkExDzANBgNVBAcMBlZpZW5uYTEaMBgG\n" - "A1UECgwRRm91bmRhdGlvbkRCLCBMTEMxGTAXBgNVBAsMEFRlc3QgZW5naW5lZXJp\n" - "bmcxFTATBgNVBAMMDE1yLiBCaWcgVHVuYTEoMCYGCSqGSIb3DQEJARYZYmlnLnR1\n" - "bmFAZm91bmRhdGlvbmRiLmNvbTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC\n" - "ggEBAKZTL2edDkiet4HBTZnjysn6gOVZH2MP02KVBIv/H7e+3w7ZOIRvcPzhZe9M\n" - "3cGH1t/pkr9DSXvzIb42EffMVlpLD2VQn2H8VC2QSdJCIQcf802u+Taf+XtW6K1h\n" - "p/YPL1uhdopUs3c1oon8ykKwnOfrQYgv5pUa7jQdMkltI2MQJU3uFq3Z/LHTvIKe\n" - "FN+bqK0iYhZthwMG7Rld4+RgKZoT4u1B6w/duEWk9KLjgs7fTf3Oe6JHCYNqwBJi\n" - "78sJalwXz9Wf8wmMaYSG0XNA7vBOdpTFhVPSsh6e3rkydf5HydMade/II98MWpMe\n" - "hFg7FFMaJP6ig8p5iL+9QP2VMCkCAwEAAaNQME4wHQYDVR0OBBYEFIXGmIcKptBP\n" - "v3i9WS/mK78o5E/MMB8GA1UdIwQYMBaAFIXGmIcKptBPv3i9WS/mK78o5E/MMAwG\n" - "A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQEFBQADggEBAJkVgNGOXT+ZHCNEYLjr/6OM\n" - "UCHvwlMeaEyqxaOmK26J2kAADPhjBZ7lZOHWb2Wzb+BiQUIFGwNIMoRvsg8skpJa\n" - "OCqpVciHVXY/U8BiYY70DKozRza93Ab9om3pySGDJ/akdCjqbMT1Cb7Kloyw+hNh\n" - "XD4MML0lYiUE9KK35xyK6FgTx4A7IXl4b3lWBgglqTh4+P5J1+xy8AYJ0VfPoP7y\n" - "OoZgwAmkpkMnalReNkN7LALHGqMzv/qH04ODlkU/HUGgExtnINMxK9VEDIe/yLGm\n" - "DHy7gcQMj5Hyymack/d4ZF8CSrYpGZQeZGXoxOmTDwWcXgnYA+2o7lOYPb5Uu08=\n" - "-----END CERTIFICATE-----\n" - "-----BEGIN PRIVATE KEY-----\n" - "MIIEvwIBADANBgkqhkiG9w0BAQEFAASCBKkwggSlAgEAAoIBAQCmUy9nnQ5InreB\n" - "wU2Z48rJ+oDlWR9jD9NilQSL/x+3vt8O2TiEb3D84WXvTN3Bh9bf6ZK/Q0l78yG+\n" - "NhH3zFZaSw9lUJ9h/FQtkEnSQiEHH/NNrvk2n/l7VuitYaf2Dy9boXaKVLN3NaKJ\n" - "/MpCsJzn60GIL+aVGu40HTJJbSNjECVN7hat2fyx07yCnhTfm6itImIWbYcDBu0Z\n" - "XePkYCmaE+LtQesP3bhFpPSi44LO3039znuiRwmDasASYu/LCWpcF8/Vn/MJjGmE\n" - "htFzQO7wTnaUxYVT0rIent65MnX+R8nTGnXvyCPfDFqTHoRYOxRTGiT+ooPKeYi/\n" - "vUD9lTApAgMBAAECggEBAIYCmDtfq9aPK0P8v82yX/4FPD2OZV+nrKXNc3BpCuE9\n" - "hPOtyX/LWrol0b/Rqwr3rAWVaIt6Z4bbCuD7J9cEaL8voyP6pbCJYjmj/BbQ+VOI\n" - "Rrzcsid1Fcpu5+JqwK3c5kdp/NzQChmOuXt8lmrNal7iilZ0YdDZdfu/WnkW2mBB\n" - "oQHkujlnWr4PNYdwMOnBU6TwdOuz+inPVMLohOO0Vr585OxPsGzG2Ud3yQ/t34Cq\n" - "F9nmOXQoszftGKsL1yuh/3fGj/O86g/CRsUy05qZhDDBEYQD6qZCvD5+yp8oOWIR\n" - "SljM3GXDBnJqRPhP+Nyf6e6/GoQtfVZ9MPRzDDPzIBECgYEA2kX/zAs6taOiNqCb\n" - "6nVGe7/3uQJz/CkmOSKIFKUu7lCEUjmMYpK3Xzp26RTUR9cT+g9y+cnJO1Vbaxtf\n" - "Qidje6K+Oi1pQyUGQ6W+U8cPJHz43PVa7IB5Az5i/sS2tu0BGhvGo9G6iYQjxXeD\n" - "1197DRACgnm5AORQMum616XvSPMCgYEAwxKbkAzJzfZF6A3Ys+/0kycNfDP8xZoC\n" - "1zV3d1b2JncsdAPCHYSKtpniRrQN9ASa3RMdkh+wrMN/KlbtU9Ddoc4NHxSTFV7F\n" - "wypFMzLZslqkQ6uHnVVewHV7prfoKsMci2c9iHO7W8TEv4aqW8XDd8OozP3/q2j4\n" - "hvL7VIAVqXMCgYEAwAFnfOQ75uBkp00tGlfDgsRhc5vWz3CbMRNRRWfxGq41V+dL\n" - "uMJ7EAfr5ijue6uU5RmF+HkqzUjOvC894oGnn3CPibm8qNX+5q7799JZXa2ZdTVX\n" - "oEd7LAFLL/V3DP77Qy4/1Id/Ycydcu0pSuGw6tK0gnX06fXtHnxAYcaT8UUCgYAE\n" - "MytcP5o8r/ezVlD7Fsh6PpYAvZHMo1M6VPFchWfJTjmLyeTtA8SEx+1iPlAql8rJ\n" - "xbaWRc5k+dSMEdEMQ+vxpuELcUL1a9PwLsHMp2SefWsZ9eB2l7bxh9YAsebyvL6p\n" - "lbBydqNrB2KBCSIz1Z8uveytdS6C/0CSjzqwCA3vVwKBgQDAXqjo3xrzMlHeXm5o\n" - "qH/OjajjqbnPXHolHDitbLubyQ4E6KhMBMxfChBe/8VptB/Gs0efVbMVGuabxY7Q\n" - "iastGId8HyONy3UPGPxCn4b95cIxKvdpt+hvWtYHIBCfHXluQK7zsDMgvtXjYNiz\n" - "peZRikYlwmu1K2YRTf7oLE2Ogw==\n" - "-----END PRIVATE KEY-----\n"; -*/ - template T simulate( const T& in ) { BinaryWriter writer(AssumeVersion(currentProtocolVersion)); From 321dded7dd91f24b7838970553e6d079aefdf006 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 14 Feb 2020 16:45:04 -0800 Subject: [PATCH 18/44] rely on preverified to verify the certificate --- flow/Net2.actor.cpp | 8 +++++--- flow/TLSPolicy.cpp | 37 ++++++------------------------------- flow/TLSPolicy.h | 2 +- 3 files changed, 12 insertions(+), 35 deletions(-) diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 7c128944c0..d01ba92364 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -1399,7 +1399,7 @@ void ASIOReactor::wake() { } // namespace net2 -bool insecurely_always_accept_for_testing(bool _1, boost::asio::ssl::context* _2) { +bool insecurely_always_accept(bool _1, boost::asio::ssl::verify_context& _2) { return true; } @@ -1408,9 +1408,11 @@ INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, boo sslContext->set_options(boost::asio::ssl::context::default_workarounds); sslContext->set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); if (policy) { - sslContext->set_verify_callback([policy](bool _, boost::asio::ssl::verify_context& ctx) { - return policy->verify_peer(ctx.native_handle()); + sslContext->set_verify_callback([policy](bool preverified, boost::asio::ssl::verify_context& ctx) { + return policy->verify_peer(preverified, ctx.native_handle()); }); + } else { + sslContext->set_verify_callback(boost::bind(&insecurely_always_accept, _1, _2)); } N2::g_net2 = new N2::Net2(useThreadPool, useMetrics, sslContext, tlsPassword); } diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp index 2074e6d759..5ebd1af6a2 100644 --- a/flow/TLSPolicy.cpp +++ b/flow/TLSPolicy.cpp @@ -404,27 +404,6 @@ std::tuple check_verify(const TLSPolicy::Rule* verify, X509_ST // if returning false, give a reason string std::string reason = ""; - if (verify->verify_time) { - X509_VERIFY_PARAM* param = X509_STORE_CTX_get0_param(store_ctx); - time_t cert_time = X509_VERIFY_PARAM_get_time(param); - STACK_OF(X509) *sk = X509_STORE_CTX_get0_chain(store_ctx); - for (int i = 0; i < sk_X509_num(sk); i++) { - X509* cert = sk_X509_value(sk, i); - - int rv = X509_cmp_time(X509_get0_notBefore(cert), &cert_time); - if (rv >= 0) { - reason = "Cert is not yet valid"; - goto err; - } - - rv = X509_cmp_time(X509_get0_notAfter(cert), &cert_time); - if (rv <= 0) { - reason = "Cert has expired"; - goto err; - } - } - } - // Check subject criteria. cert = sk_X509_value(X509_STORE_CTX_get0_chain(store_ctx), 0); if ((subject = X509_get_subject_name(cert)) == NULL) { @@ -470,10 +449,7 @@ std::tuple check_verify(const TLSPolicy::Rule* verify, X509_ST return std::make_tuple(rc, reason); } -bool TLSPolicy::verify_peer(X509_STORE_CTX* store_ctx) { - //??? - //X509_STORE_CTX* store_ctx = X509_STORE_CTX_new(); - //X509_STORE_CTX_init(store_ctx, X509_STORE_CTX_get0_store(store_ctx_), X509_STORE_CTX_get0_cert(store_ctx_), X509_STORE_CTX_get0_chain(store_ctx_)); +bool TLSPolicy::verify_peer(bool preverified, X509_STORE_CTX* store_ctx) { bool rc = false; std::set verify_failure_reasons; bool verify_success; @@ -483,15 +459,14 @@ bool TLSPolicy::verify_peer(X509_STORE_CTX* store_ctx) { if (std::any_of(rules.begin(), rules.end(), [](const Rule& r){ return r.verify_cert; })) return true; - X509_STORE_CTX_set_default(store_ctx, is_client ? "ssl_server" : "ssl_client"); - X509_VERIFY_PARAM_set_flags(X509_STORE_CTX_get0_param(store_ctx), X509_V_FLAG_NO_CHECK_TIME); - if (X509_verify_cert(store_ctx) <= 0) { - const char *errstr = X509_verify_cert_error_string(X509_STORE_CTX_get_error(store_ctx)); - verify_failure_reason = "Verify cert error: " + std::string(errstr); - TraceEvent("TLSPolicyFailure").suppressFor(1.0).detail("Reason", verify_failure_reason); + if(!preverified) { return false; } + if(!rules.size()) { + return true; + } + // Any matching rule is sufficient. for (auto &verify_rule: rules) { std::tie(verify_success, verify_failure_reason) = check_verify(&verify_rule, store_ctx, is_client); diff --git a/flow/TLSPolicy.h b/flow/TLSPolicy.h index a027ae2657..2ab5aea77f 100644 --- a/flow/TLSPolicy.h +++ b/flow/TLSPolicy.h @@ -70,7 +70,7 @@ public: }; bool set_verify_peers(std::vector verify_peers); - bool verify_peer(X509_STORE_CTX* store_ctx); + bool verify_peer(bool preverified, X509_STORE_CTX* store_ctx); TLSPolicy(Is client) : is_client(client == Is::CLIENT) {} virtual ~TLSPolicy(); From 693e46900319dbe0eab7a897fde0352094d17fc3 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 14 Feb 2020 16:49:52 -0800 Subject: [PATCH 19/44] Changed the handshake lock to a BoundedFlowLock, which will enforce that old handshakes complete before starting to initiate new handshakes --- flow/Knobs.cpp | 3 +- flow/Knobs.h | 3 +- flow/Net2.actor.cpp | 10 ++-- flow/genericactors.actor.h | 105 +++++++++++++++++++++++++++++++++++++ flow/network.cpp | 2 +- flow/network.h | 4 +- 6 files changed, 117 insertions(+), 10 deletions(-) diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index da489ac51b..7dd802e0d6 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -124,7 +124,8 @@ FlowKnobs::FlowKnobs(bool randomize, bool isSimulated) { init( MAX_PACKET_SEND_BYTES, 256 * 1024 ); init( MIN_PACKET_BUFFER_BYTES, 4 * 1024 ); init( MIN_PACKET_BUFFER_FREE_BYTES, 256 ); - init( HANDSHAKE_LIMIT, 200 ); + init( UNRESTRICTED_HANDSHAKE_LIMIT, 15 ); + init( BOUNDED_HANDSHAKE_LIMIT, 400 ); //Sim2 init( MIN_OPEN_TIME, 0.0002 ); diff --git a/flow/Knobs.h b/flow/Knobs.h index ca4f64d40d..fc72b3cf44 100644 --- a/flow/Knobs.h +++ b/flow/Knobs.h @@ -145,7 +145,8 @@ public: int MAX_PACKET_SEND_BYTES; int MIN_PACKET_BUFFER_BYTES; int MIN_PACKET_BUFFER_FREE_BYTES; - int HANDSHAKE_LIMIT; + int UNRESTRICTED_HANDSHAKE_LIMIT; + int BOUNDED_HANDSHAKE_LIMIT; //Sim2 //FIMXE: more parameters could be factored out diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index d01ba92364..7ce1824f5d 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -513,8 +513,8 @@ public: } } - wait(g_network->networkInfo.handshakeLock->take()); - state FlowLock::Releaser releaser(*g_network->networkInfo.handshakeLock); + int64_t permitNumber = wait(g_network->networkInfo.handshakeLock->take()); + state BoundedFlowLock::Releaser releaser(g_network->networkInfo.handshakeLock, permitNumber); BindPromise p("N2_AcceptHandshakeError", UID()); auto onHandshook = p.getFuture(); @@ -531,7 +531,6 @@ public: } self->closeSocket(); connected.sendError(connection_failed()); - throw; } } @@ -554,8 +553,8 @@ public: ACTOR static void doConnectHandshake( Reference self, Promise connected) { try { - wait(g_network->networkInfo.handshakeLock->take()); - state FlowLock::Releaser releaser(*g_network->networkInfo.handshakeLock); + int64_t permitNumber = wait(g_network->networkInfo.handshakeLock->take()); + state BoundedFlowLock::Releaser releaser(g_network->networkInfo.handshakeLock, permitNumber); BindPromise p("N2_ConnectHandshakeError", self->id); Future onHandshook = p.getFuture(); @@ -573,7 +572,6 @@ public: } self->closeSocket(); connected.sendError(connection_failed()); - throw; } } diff --git a/flow/genericactors.actor.h b/flow/genericactors.actor.h index ecc937e6c4..66d6dd99ca 100644 --- a/flow/genericactors.actor.h +++ b/flow/genericactors.actor.h @@ -32,6 +32,7 @@ #include "flow/flow.h" #include "flow/Knobs.h" #include "flow/Util.h" +#include "flow/IndexedSet.h" #include "flow/actorcompiler.h" // This must be the last #include. #pragma warning( disable: 4355 ) // 'this' : used in base member initializer list @@ -1339,6 +1340,110 @@ private: } }; +struct NotifiedInt { + NotifiedInt( int64_t val = 0 ) : val(val) {} + + Future whenAtLeast( int64_t limit ) { + if (val >= limit) + return Void(); + Promise p; + waiting.push( std::make_pair(limit,p) ); + return p.getFuture(); + } + + int64_t get() const { return val; } + + void set( int64_t v ) { + ASSERT( v >= val ); + if (v != val) { + val = v; + + std::vector> toSend; + while ( waiting.size() && v >= waiting.top().first ) { + Promise p = std::move(waiting.top().second); + waiting.pop(); + toSend.push_back(p); + } + for(auto& p : toSend) { + p.send(Void()); + } + } + } + + void operator=( int64_t v ) { + set( v ); + } + + NotifiedInt(NotifiedInt&& r) BOOST_NOEXCEPT : waiting(std::move(r.waiting)), val(r.val) {} + void operator=(NotifiedInt&& r) BOOST_NOEXCEPT { waiting = std::move(r.waiting); val = r.val; } + +private: + typedef std::pair> Item; + struct ItemCompare { + bool operator()(const Item& a, const Item& b) { return a.first > b.first; } + }; + std::priority_queue, ItemCompare> waiting; + int64_t val; +}; + +struct BoundedFlowLock : NonCopyable, public ReferenceCounted { + // BoundedFlowLock is different from a FlowLock in that it has a bound on how many locks can be taken from the oldest outstanding lock. + // For instance, with a FlowLock that has two permits, if one permit is taken but never released, the other permit can be reused an unlimited + // amount of times, but with a BoundedFlowLock, it can only be reused a fixed number of times. + + struct Releaser : NonCopyable { + BoundedFlowLock* lock; + int64_t permitNumber; + Releaser() : lock(nullptr), permitNumber(0) {} + Releaser( BoundedFlowLock* lock, int64_t permitNumber ) : lock(lock), permitNumber(permitNumber) {} + Releaser(Releaser&& r) BOOST_NOEXCEPT : lock(r.lock), permitNumber(r.permitNumber) { r.permitNumber = 0; } + void operator=(Releaser&& r) { if (permitNumber) lock->release(permitNumber); lock = r.lock; permitNumber = r.permitNumber; r.permitNumber = 0; } + + void release() { + if (permitNumber) { + lock->release(permitNumber); + } + permitNumber = 0; + } + + ~Releaser() { if (permitNumber) lock->release(permitNumber); } + }; + + BoundedFlowLock() : unrestrictedPermits(1), boundedPermits(0), nextPermitNumber(0), minOutstanding(0) {} + explicit BoundedFlowLock(int64_t unrestrictedPermits, int64_t boundedPermits) : unrestrictedPermits(unrestrictedPermits), boundedPermits(boundedPermits), nextPermitNumber(0), minOutstanding(0) {} + + Future take() { + return takeActor(this); + } + void release( int64_t permitNumber ) { + outstanding.erase(permitNumber); + updateMinOutstanding(); + } +private: + IndexedSet outstanding; + NotifiedInt minOutstanding; + int64_t nextPermitNumber; + const int64_t unrestrictedPermits; + const int64_t boundedPermits; + + void updateMinOutstanding() { + auto it = outstanding.index(unrestrictedPermits-1); + if(it == outstanding.end()) { + minOutstanding.set(nextPermitNumber); + } else { + minOutstanding.set(*it); + } + } + + ACTOR static Future takeActor(BoundedFlowLock* lock) { + state int64_t permitNumber = ++lock->nextPermitNumber; + lock->outstanding.insert(permitNumber, 1); + lock->updateMinOutstanding(); + wait( lock->minOutstanding.whenAtLeast(std::max(0, permitNumber - lock->boundedPermits)) ); + return permitNumber; + } +}; + ACTOR template Future yieldPromiseStream( FutureStream input, PromiseStream output, TaskPriority taskID = TaskPriority::DefaultYield ) { loop { diff --git a/flow/network.cpp b/flow/network.cpp index 1b7760a45d..eae57a8b9a 100644 --- a/flow/network.cpp +++ b/flow/network.cpp @@ -201,4 +201,4 @@ TEST_CASE("/flow/network/ipaddress") { return Void(); } -NetworkInfo::NetworkInfo() : handshakeLock( new FlowLock(FLOW_KNOBS->HANDSHAKE_LIMIT) ) {} +NetworkInfo::NetworkInfo() : handshakeLock( new BoundedFlowLock(FLOW_KNOBS->UNRESTRICTED_HANDSHAKE_LIMIT, FLOW_KNOBS->BOUNDED_HANDSHAKE_LIMIT) ) {} diff --git a/flow/network.h b/flow/network.h index c7d292b747..95f1ddeb4b 100644 --- a/flow/network.h +++ b/flow/network.h @@ -329,6 +329,8 @@ struct NetworkMetrics { NetworkMetrics() {} }; +struct BoundedFlowLock; + struct NetworkInfo { NetworkMetrics metrics; double oldestAlternativesFailure = 0; @@ -336,7 +338,7 @@ struct NetworkInfo { double lastAlternativesFailureSkipDelay = 0; std::map, std::pair> serverTLSConnectionThrottler; - struct FlowLock* handshakeLock; + BoundedFlowLock* handshakeLock; NetworkInfo(); }; From 663d176fdb6aa3679ff2e32b1f8015707ebda247 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 14 Feb 2020 16:50:55 -0800 Subject: [PATCH 20/44] fix: coordinators auto could added 0.0.0.0:0 as a coordinator --- fdbclient/ManagementAPI.actor.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 6311bb0c06..e57ae9bc6e 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -1120,8 +1120,7 @@ struct AutoQuorumChange : IQuorumChange { *err = CoordinatorsResult::NOT_ENOUGH_MACHINES; return vector(); } - desiredCount = std::max(oldCoordinators.size(), (workers.size() - 1) | 1); - chosen.resize(desiredCount); + chosen.resize((chosen.size() - 1) | 1); } return chosen; From 88d36af9c70c1fb624152bbdb037a6f9e06dc79c Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Wed, 19 Feb 2020 00:57:05 -0800 Subject: [PATCH 21/44] Fix --tls_password and add better error logging This refactors all tls settings into a TLSParams object so that we can set the password before loading any certificates. It turns out that the FDBLibTLS code did really nice things with error logging, but I just didn't understand openssl enough before to realize what pieces I should be copying. --- fdbclient/NativeAPI.actor.cpp | 21 ++++++++---------- fdbserver/fdbserver.actor.cpp | 22 +++++-------------- flow/Net2.actor.cpp | 36 +++++++++++++++++++++++++----- flow/TLSPolicy.cpp | 41 ++++++++++++++++++++++++++++++++++- flow/TLSPolicy.h | 18 ++++++++++++--- flow/network.h | 2 +- 6 files changed, 101 insertions(+), 39 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index f8673cd661..ec5f6ed213 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -70,6 +70,7 @@ using std::pair; #define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) NetworkOptions networkOptions; +TLSParams tlsParams; static Reference tlsPolicy; static void initTLSPolicy() { @@ -890,38 +891,34 @@ void setNetworkOption(FDBNetworkOptions::Option option, Optional valu break; case FDBNetworkOptions::TLS_CERT_PATH: validateOptionValue(value, true); - networkOptions.sslContext.use_certificate_chain_file(value.get().toString()); + tlsParams.tlsCertPath = value.get().toString(); break; case FDBNetworkOptions::TLS_CERT_BYTES: { validateOptionValue(value, true); - std::string cert = value.get().toString(); - networkOptions.sslContext.use_certificate(boost::asio::buffer(cert.data(), cert.size()), boost::asio::ssl::context::pem); + tlsParams.tlsCertBytes = value.get().toString(); break; } case FDBNetworkOptions::TLS_CA_PATH: { validateOptionValue(value, true); - std::string cert = readFileBytes(value.get().toString(), CERT_FILE_MAX_SIZE); - networkOptions.sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); + tlsParams.tlsCAPath = value.get().toString(); break; } case FDBNetworkOptions::TLS_CA_BYTES: { validateOptionValue(value, true); - std::string cert = value.get().toString(); - networkOptions.sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); + tlsParams.tlsCABytes = value.get().toString(); break; } case FDBNetworkOptions::TLS_PASSWORD: validateOptionValue(value, true); - networkOptions.tlsPassword = value.get().toString(); + tlsParams.tlsPassword = value.get().toString(); break; case FDBNetworkOptions::TLS_KEY_PATH: validateOptionValue(value, true); - networkOptions.sslContext.use_private_key_file(value.get().toString(), boost::asio::ssl::context::pem); + tlsParams.tlsKeyPath = value.get().toString(); break; case FDBNetworkOptions::TLS_KEY_BYTES: { validateOptionValue(value, true); - std::string cert = value.get().toString(); - networkOptions.sslContext.use_private_key(boost::asio::buffer(cert.data(), cert.size()), boost::asio::ssl::context::pem); + tlsParams.tlsKeyBytes = value.get().toString(); break; } case FDBNetworkOptions::TLS_VERIFY_PEERS: @@ -991,7 +988,7 @@ void setupNetwork(uint64_t transportId, bool useMetrics) { initTLSPolicy(); - g_network = newNet2(&networkOptions.sslContext, false, useMetrics || networkOptions.traceDirectory.present(), tlsPolicy, networkOptions.tlsPassword); + g_network = newNet2(&networkOptions.sslContext, false, useMetrics || networkOptions.traceDirectory.present(), tlsPolicy, tlsParams); FlowTransport::createInstance(true, transportId); Net2FileSystem::newFileSystem(); } diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index f7c25d7d8a..8c4979057c 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -966,7 +966,7 @@ int main(int argc, char* argv[]) { boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); Reference tlsPolicy = Reference(new TLSPolicy(TLSPolicy::Is::SERVER)); - std::string tlsCertPath, tlsKeyPath, tlsCAPath, tlsPassword; + TLSParams tlsParams; std::vector tlsVerifyPeers; double fileIoTimeout = 0.0; bool fileIoWarnOnly = false; @@ -1339,16 +1339,16 @@ int main(int argc, char* argv[]) { args.OptionArg(); break; case TLSOptions::OPT_TLS_CERTIFICATES: - tlsCertPath = args.OptionArg(); + tlsParams.tlsCertPath = args.OptionArg(); break; case TLSOptions::OPT_TLS_PASSWORD: - tlsPassword = args.OptionArg(); + tlsParams.tlsPassword = args.OptionArg(); break; case TLSOptions::OPT_TLS_CA_FILE: - tlsCAPath = args.OptionArg(); + tlsParams.tlsCAPath = args.OptionArg(); break; case TLSOptions::OPT_TLS_KEY: - tlsKeyPath = args.OptionArg(); + tlsParams.tlsKeyPath = args.OptionArg(); break; case TLSOptions::OPT_TLS_VERIFY_PEERS: tlsVerifyPeers.push_back(args.OptionArg()); @@ -1556,21 +1556,11 @@ int main(int argc, char* argv[]) { openTraceFile(NetworkAddress(), rollsize, maxLogsSize, logFolder, "trace", logGroup); } else { #ifndef TLS_DISABLED - if ( tlsCertPath.size() ) { - sslContext.use_certificate_chain_file(tlsCertPath); - } - if (tlsCAPath.size()) { - std::string cert = readFileBytes(tlsCAPath, CERT_FILE_MAX_SIZE); - sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); - } - if (tlsKeyPath.size()) { - sslContext.use_private_key_file(tlsKeyPath, boost::asio::ssl::context::pem); - } if ( tlsVerifyPeers.size() ) { tlsPolicy->set_verify_peers( tlsVerifyPeers ); } #endif - g_network = newNet2(&sslContext, useThreadPool, true, tlsPolicy, tlsPassword); + g_network = newNet2(&sslContext, useThreadPool, true, tlsPolicy, tlsParams); FlowTransport::createInstance(false, 1); const bool expectsPublicAddress = (role == FDBD || role == NetworkTestServer || role == Restore); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 7ce1824f5d..8547f74e29 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -113,7 +113,7 @@ thread_local INetwork* thread_network = 0; class Net2 sealed : public INetwork, public INetworkConnections { public: - Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, std::string tlsPassword); + Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, const TLSParams& tlsParams); void run(); void initMetrics(); @@ -253,7 +253,7 @@ public: try { if (error) { // Log the error... - TraceEvent(SevWarn, errContext, errID).suppressFor(1.0).detail("Message", error.value()); + TraceEvent(SevWarn, errContext, errID).suppressFor(1.0).detail("Message", error.value()).detail("WhichMeans", TLSPolicy::ErrorString(error)); p.sendError( connection_failed() ); } else p.send( Void() ); @@ -828,7 +828,11 @@ struct PromiseTask : public Task, public FastAllocated { } }; -Net2::Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, std::string tlsPassword) +// 5MB for loading files into memory +// TODO: Move to a headerfile and delete all the copies of this. +#define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) + +Net2::Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, const TLSParams& tlsParams) : useThreadPool(useThreadPool), network(this), reactor(this), @@ -839,12 +843,32 @@ Net2::Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslCo lastMinTaskID(TaskPriority::Zero), numYields(0), sslContext(sslContext), - tlsPassword(tlsPassword) + tlsPassword(tlsParams.tlsPassword) { TraceEvent("Net2Starting"); if(sslContext) { sslContext->set_password_callback(std::bind(&Net2::get_password, this)); + + if (tlsParams.tlsCertPath.size() ) { + sslContext->use_certificate_chain_file(tlsParams.tlsCertPath); + } + if (tlsParams.tlsCertBytes.size() ) { + sslContext->use_certificate(boost::asio::buffer(tlsParams.tlsCertBytes.data(), tlsParams.tlsCertBytes.size()), boost::asio::ssl::context::pem); + } + if (tlsParams.tlsCAPath.size()) { + std::string cert = readFileBytes(tlsParams.tlsCAPath, CERT_FILE_MAX_SIZE); + sslContext->add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); + } + if (tlsParams.tlsCABytes.size()) { + sslContext->add_certificate_authority(boost::asio::buffer(tlsParams.tlsCABytes.data(), tlsParams.tlsCABytes.size())); + } + if (tlsParams.tlsKeyPath.size()) { + sslContext->use_private_key_file(tlsParams.tlsKeyPath, boost::asio::ssl::context::pem); + } + if (tlsParams.tlsKeyBytes.size()) { + sslContext->use_private_key(boost::asio::buffer(tlsParams.tlsKeyBytes.data(), tlsParams.tlsKeyBytes.size()), boost::asio::ssl::context::pem); + } } // Set the global members @@ -1401,7 +1425,7 @@ bool insecurely_always_accept(bool _1, boost::asio::ssl::verify_context& _2) { return true; } -INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, bool useMetrics, Reference policy, std::string tlsPassword) { +INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, bool useMetrics, Reference policy, const TLSParams& tlsParams) { try { sslContext->set_options(boost::asio::ssl::context::default_workarounds); sslContext->set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); @@ -1412,7 +1436,7 @@ INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, boo } else { sslContext->set_verify_callback(boost::bind(&insecurely_always_accept, _1, _2)); } - N2::g_net2 = new N2::Net2(useThreadPool, useMetrics, sslContext, tlsPassword); + N2::g_net2 = new N2::Net2(useThreadPool, useMetrics, sslContext, tlsParams); } catch(boost::system::system_error e) { TraceEvent("Net2InitError").detail("Message", e.what()); diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp index 5ebd1af6a2..e636c4af8d 100644 --- a/flow/TLSPolicy.cpp +++ b/flow/TLSPolicy.cpp @@ -37,9 +37,46 @@ #include "flow/Trace.h" #include "flow/TLSPolicy.h" +std::string TLSPolicy::ErrorString(boost::system::error_code e) { + char* str = ERR_error_string(e.value(), NULL); + return std::string(str); +} + // To force typeinfo to only be emitted once. TLSPolicy::~TLSPolicy() {} +std::string TLSPolicy::toString() const { + std::stringstream ss; + ss << "TLSPolicy{ Rules=["; + for (const auto &r : rules) { + ss << " " << r.toString() << ","; + } + ss << " ] }"; + return ss.str(); +} + +std::string TLSPolicy::Rule::toString() const { + std::stringstream ss; + + ss << "Rule{ verify_cert=" << verify_cert + << ", verify_time=" << verify_time; + ss << ", Subject=["; + for (const auto& s : subject_criteria) { + ss << " { NID=" << s.first << ", Criteria=" << s.second.criteria << "},"; + } + ss << " ], Issuer=["; + for (const auto& s : issuer_criteria) { + ss << " { NID=" << s.first << ", Criteria=" << s.second.criteria << "},"; + } + ss << " ], Root=["; + for (const auto& s : root_criteria) { + ss << " { NID=" << s.first << ", Criteria=" << s.second.criteria << "},"; + } + ss << " ] }"; + + return ss.str(); +} + static int hexValue(char c) { static char const digits[] = "0123456789ABCDEF"; @@ -456,10 +493,12 @@ bool TLSPolicy::verify_peer(bool preverified, X509_STORE_CTX* store_ctx) { std::string verify_failure_reason; // If certificate verification is disabled, there's nothing more to do. - if (std::any_of(rules.begin(), rules.end(), [](const Rule& r){ return r.verify_cert; })) + if (std::any_of(rules.begin(), rules.end(), [](const Rule& r){ return !r.verify_cert; })) { return true; + } if(!preverified) { + TraceEvent("TLSPolicyFailure").suppressFor(1.0).detail("Reason", "preverification failed").detail("VerifyError", X509_verify_cert_error_string(X509_STORE_CTX_get_error(store_ctx))); return false; } diff --git a/flow/TLSPolicy.h b/flow/TLSPolicy.h index 2ab5aea77f..e440176c30 100644 --- a/flow/TLSPolicy.h +++ b/flow/TLSPolicy.h @@ -26,8 +26,14 @@ #include #include #include +#include #include "flow/FastRef.h" +struct TLSParams { + std::string tlsCertPath, tlsKeyPath, tlsCAPath, tlsPassword; + std::string tlsCertBytes, tlsKeyBytes, tlsCABytes; +}; + typedef int NID; enum class MatchType { @@ -69,18 +75,24 @@ public: SERVER }; - bool set_verify_peers(std::vector verify_peers); - bool verify_peer(bool preverified, X509_STORE_CTX* store_ctx); - TLSPolicy(Is client) : is_client(client == Is::CLIENT) {} virtual ~TLSPolicy(); virtual void addref() { ReferenceCounted::addref(); } virtual void delref() { ReferenceCounted::delref(); } + static std::string ErrorString(boost::system::error_code e); + + bool set_verify_peers(std::vector verify_peers); + bool verify_peer(bool preverified, X509_STORE_CTX* store_ctx); + + std::string toString() const; + struct Rule { explicit Rule(std::string input); + std::string toString() const; + std::map< NID, Criteria > subject_criteria; std::map< NID, Criteria > issuer_criteria; std::map< NID, Criteria > root_criteria; diff --git a/flow/network.h b/flow/network.h index 95f1ddeb4b..8df670307c 100644 --- a/flow/network.h +++ b/flow/network.h @@ -406,7 +406,7 @@ typedef NetworkAddressList (*NetworkAddressesFuncPtr)(); class INetwork; extern INetwork* g_network; -extern INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool = false, bool useMetrics = false, Reference policy = Reference(), std::string tlsPassword = ""); +extern INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool = false, bool useMetrics = false, Reference policy = Reference(), const TLSParams& tlsParams = TLSParams()); class INetwork { public: From 3b9c38f40d239946a6c2985268de3d2fae46befa Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Wed, 19 Feb 2020 14:31:51 -0800 Subject: [PATCH 22/44] Fix fdb_flow_tester linker error --- bindings/flow/tester/local.mk | 1 + 1 file changed, 1 insertion(+) diff --git a/bindings/flow/tester/local.mk b/bindings/flow/tester/local.mk index 83444774bd..3bfc3e4be6 100644 --- a/bindings/flow/tester/local.mk +++ b/bindings/flow/tester/local.mk @@ -23,6 +23,7 @@ fdb_flow_tester_CFLAGS := -Ibindings/c $(fdbrpc_CFLAGS) fdb_flow_tester_LDFLAGS := -Llib $(fdbrpc_LDFLAGS) -lfdb_c fdb_flow_tester_LIBS := lib/libfdb_flow.a lib/libflow.a lib/libfdb_c.$(DLEXT) +fdb_flow_tester_STATIC_LIBS := $(TLS_LIBS) fdb_flow_tester: lib/libfdb_c.$(DLEXT) @mkdir -p bindings/flow/bin From e06c3e2eb7457f19a1ed98a67899058d359bb13f Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 19 Feb 2020 15:10:54 -0800 Subject: [PATCH 23/44] fix: checkForExcludedServer needs to check both the tls and non-tls address --- fdbclient/ManagementAPI.actor.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index e57ae9bc6e..e2bd6648d2 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -1494,10 +1494,14 @@ ACTOR Future> checkForExcludingServers(Database cx, vec state bool ok = true; inProgressExclusion.clear(); for(auto& s : serverList) { - auto addr = decodeServerListValue( s.value ).address(); - if ( addressExcluded(exclusions, addr) ) { + auto addresses = decodeServerListValue( s.value ).getKeyValues.getEndpoint().addresses; + if ( addressExcluded(exclusions, addresses.address) ) { ok = false; - inProgressExclusion.insert(addr); + inProgressExclusion.insert(addresses.address); + } + if ( addresses.secondaryAddress.present() && addressExcluded(exclusions, addresses.secondaryAddress.get()) ) { + ok = false; + inProgressExclusion.insert(addresses.secondaryAddress.get()); } } From 3bcecea30b5ace923cfc56fba063bd1946247eb2 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Wed, 19 Feb 2020 17:01:36 -0800 Subject: [PATCH 24/44] Fix the cmake build when neither SSL library is found. --- cmake/FDBComponents.cmake | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmake/FDBComponents.cmake b/cmake/FDBComponents.cmake index b425b495f2..d6fe00a598 100644 --- a/cmake/FDBComponents.cmake +++ b/cmake/FDBComponents.cmake @@ -21,7 +21,9 @@ else() if(NOT OPENSSL_FOUND) set(LIBRESSL_USE_STATIC_LIBS TRUE) find_package(LibreSSL) - add_library(OpenSSL::SSL ALIAS LibreSSL) + if (LIBRESSL_FOUND) + add_library(OpenSSL::SSL ALIAS LibreSSL) + endif() endif() if(OPENSSL_FOUND OR LIBRESSL_FOUND) set(WITH_TLS ON) From 761da5a0592b01f0d3891c7423ab3747a1b48ec6 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 19 Feb 2020 17:59:45 -0800 Subject: [PATCH 25/44] code cleanup --- bindings/flow/fdb_flow.actor.cpp | 3 +- bindings/flow/tester/Tester.actor.cpp | 6 +- fdbbackup/backup.actor.cpp | 13 +- fdbcli/fdbcli.actor.cpp | 13 +- fdbclient/NativeAPI.actor.cpp | 3 +- fdbclient/NativeAPI.actor.h | 4 +- fdbrpc/CMakeLists.txt | 1 - fdbrpc/ITLSPlugin.h | 136 ------- fdbrpc/TLSConnection.actor.cpp | 539 -------------------------- fdbrpc/TLSConnection.h | 175 --------- fdbrpc/fdbrpc.vcxproj | 3 - fdbrpc/fdbrpc.vcxproj.filters | 3 - fdbrpc/sim2.actor.cpp | 5 +- fdbserver/SimulatedCluster.h | 2 - fdbserver/fdbserver.actor.cpp | 16 +- flow/Net2.actor.cpp | 86 ++-- flow/TLSPolicy.h | 32 ++ flow/network.h | 2 +- 18 files changed, 102 insertions(+), 940 deletions(-) delete mode 100644 fdbrpc/ITLSPlugin.h delete mode 100644 fdbrpc/TLSConnection.actor.cpp delete mode 100644 fdbrpc/TLSConnection.h diff --git a/bindings/flow/fdb_flow.actor.cpp b/bindings/flow/fdb_flow.actor.cpp index be70d187c5..dc37e28b23 100644 --- a/bindings/flow/fdb_flow.actor.cpp +++ b/bindings/flow/fdb_flow.actor.cpp @@ -82,8 +82,7 @@ void fdb_flow_test() { fdb->setupNetwork(); startThread(networkThread, fdb); - boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); - g_network = newNet2( &sslContext, false ); + g_network = newNet2(false); openTraceFile(NetworkAddress(), 1000000, 1000000, "."); systemMonitor(); diff --git a/bindings/flow/tester/Tester.actor.cpp b/bindings/flow/tester/Tester.actor.cpp index 3011aa9987..52d193320e 100644 --- a/bindings/flow/tester/Tester.actor.cpp +++ b/bindings/flow/tester/Tester.actor.cpp @@ -1748,8 +1748,7 @@ ACTOR void startTest(std::string clusterFilename, StringRef prefix, int apiVersi populateOpsThatCreateDirectories(); // FIXME // This is "our" network - boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); - g_network = newNet2(&sslContext, false); + g_network = newNet2(false); ASSERT(!API::isAPIVersionSelected()); try { @@ -1792,8 +1791,7 @@ ACTOR void startTest(std::string clusterFilename, StringRef prefix, int apiVersi ACTOR void _test_versionstamp() { try { - boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); - g_network = newNet2(&sslContext, false); + g_network = newNet2(false); API *fdb = FDB::API::selectAPIVersion(620); diff --git a/fdbbackup/backup.actor.cpp b/fdbbackup/backup.actor.cpp index c8acb1766e..d120fa06da 100644 --- a/fdbbackup/backup.actor.cpp +++ b/fdbbackup/backup.actor.cpp @@ -38,7 +38,6 @@ #include "fdbclient/json_spirit/json_spirit_writer_template.h" #include "fdbrpc/Platform.h" -#include "fdbrpc/TLSConnection.h" #include #include @@ -3072,22 +3071,22 @@ int main(int argc, char* argv[]) { blobCredentials.push_back(args->OptionArg()); break; #ifndef TLS_DISABLED - case TLSOptions::OPT_TLS_PLUGIN: + case TLSParams::OPT_TLS_PLUGIN: args->OptionArg(); break; - case TLSOptions::OPT_TLS_CERTIFICATES: + case TLSParams::OPT_TLS_CERTIFICATES: tlsCertPath = args->OptionArg(); break; - case TLSOptions::OPT_TLS_PASSWORD: + case TLSParams::OPT_TLS_PASSWORD: tlsPassword = args->OptionArg(); break; - case TLSOptions::OPT_TLS_CA_FILE: + case TLSParams::OPT_TLS_CA_FILE: tlsCAPath = args->OptionArg(); break; - case TLSOptions::OPT_TLS_KEY: + case TLSParams::OPT_TLS_KEY: tlsKeyPath = args->OptionArg(); break; - case TLSOptions::OPT_TLS_VERIFY_PEERS: + case TLSParams::OPT_TLS_VERIFY_PEERS: tlsVerifyPeers = args->OptionArg(); break; #endif diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index fd2ac42f45..baf38a2afd 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -33,7 +33,6 @@ #include "flow/DeterministicRandom.h" #include "flow/SignalSafeUnwind.h" -#include "fdbrpc/TLSConnection.h" #include "fdbrpc/Platform.h" #include "flow/SimpleOpt.h" @@ -2503,22 +2502,22 @@ struct CLIOptions { #ifndef TLS_DISABLED // TLS Options - case TLSOptions::OPT_TLS_PLUGIN: + case TLSParams::OPT_TLS_PLUGIN: args.OptionArg(); break; - case TLSOptions::OPT_TLS_CERTIFICATES: + case TLSParams::OPT_TLS_CERTIFICATES: tlsCertPath = args.OptionArg(); break; - case TLSOptions::OPT_TLS_CA_FILE: + case TLSParams::OPT_TLS_CA_FILE: tlsCAPath = args.OptionArg(); break; - case TLSOptions::OPT_TLS_KEY: + case TLSParams::OPT_TLS_KEY: tlsKeyPath = args.OptionArg(); break; - case TLSOptions::OPT_TLS_PASSWORD: + case TLSParams::OPT_TLS_PASSWORD: tlsPassword = args.OptionArg(); break; - case TLSOptions::OPT_TLS_VERIFY_PEERS: + case TLSParams::OPT_TLS_VERIFY_PEERS: tlsVerifyPeers = args.OptionArg(); break; #endif diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ec5f6ed213..a0863ce0db 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -38,7 +38,6 @@ #include "fdbrpc/LoadBalance.h" #include "fdbrpc/Net2FileSystem.h" #include "fdbrpc/simulator.h" -#include "fdbrpc/TLSConnection.h" #include "flow/ActorCollection.h" #include "flow/DeterministicRandom.h" #include "flow/Knobs.h" @@ -988,7 +987,7 @@ void setupNetwork(uint64_t transportId, bool useMetrics) { initTLSPolicy(); - g_network = newNet2(&networkOptions.sslContext, false, useMetrics || networkOptions.traceDirectory.present(), tlsPolicy, tlsParams); + g_network = newNet2(false, useMetrics || networkOptions.traceDirectory.present(), tlsPolicy, tlsParams); FlowTransport::createInstance(true, transportId); Net2FileSystem::newFileSystem(); } diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index b480ef2520..f02be1b5b3 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -61,14 +61,12 @@ struct NetworkOptions { Optional logClientInfo; Standalone> supportedVersions; bool slowTaskProfilingEnabled; - boost::asio::ssl::context sslContext; - std::string tlsPassword; // The default values, TRACE_DEFAULT_ROLL_SIZE and TRACE_DEFAULT_MAX_LOGS_SIZE are located in Trace.h. NetworkOptions() : localAddress(""), clusterFile(""), traceDirectory(Optional()), traceRollSize(TRACE_DEFAULT_ROLL_SIZE), traceMaxLogsSize(TRACE_DEFAULT_MAX_LOGS_SIZE), traceLogGroup("default"), - traceFormat("xml"), slowTaskProfilingEnabled(false), sslContext(boost::asio::ssl::context(boost::asio::ssl::context::tlsv12)), tlsPassword("") {} + traceFormat("xml"), slowTaskProfilingEnabled(false) {} }; class Database { diff --git a/fdbrpc/CMakeLists.txt b/fdbrpc/CMakeLists.txt index e5d3a1fc3f..3aae625727 100644 --- a/fdbrpc/CMakeLists.txt +++ b/fdbrpc/CMakeLists.txt @@ -27,7 +27,6 @@ set(FDBRPC_SRCS sim2.actor.cpp sim_validation.cpp TimedRequest.h - TLSConnection.actor.cpp TraceFileIO.cpp) set(FDBRPC_THIRD_PARTY_SRCS diff --git a/fdbrpc/ITLSPlugin.h b/fdbrpc/ITLSPlugin.h deleted file mode 100644 index 4ceeb6414d..0000000000 --- a/fdbrpc/ITLSPlugin.h +++ /dev/null @@ -1,136 +0,0 @@ -/* -* ITLSPlugin.h -* -* This source file is part of the FoundationDB open source project -* -* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors -* -* Licensed 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. -*/ - -#ifndef FDB_ITLSPLUGIN_H -#define FDB_ITLSPLUGIN_H - -#pragma once - -#include - -struct ITLSSession { - enum { SUCCESS = 0, WANT_READ = -1, WANT_WRITE = -2, FAILED = -3 }; - - virtual void addref() = 0; - virtual void delref() = 0; - - // handshake should return SUCCESS if the handshake is complete, - // FAILED on fatal error, or one of WANT_READ or WANT_WRITE if the - // handshake should be reattempted after more data can be - // read/written on the underlying connection. - virtual int handshake() = 0; - - // read should return the (non-zero) number of bytes read, - // WANT_READ or WANT_WRITE if the operation is blocked by the - // underlying stream, or FAILED if there is an error (including a - // closed connection). - virtual int read(uint8_t* data, int length) = 0; - - // write should return the (non-zero) number of bytes written, or - // WANT_READ or WANT_WRITE if the operation is blocked by the - // underlying stream, or FAILED if there is an error. - virtual int write(const uint8_t* data, int length) = 0; -}; - -// Returns the number of bytes sent (possibly 0), or -1 on error -// (including connection close) -typedef int(*TLSSendCallbackFunc)(void* ctx, const uint8_t* buf, int len); - -// Returns the number of bytes read (possibly 0), or -1 on error -// (including connection close) -typedef int(*TLSRecvCallbackFunc)(void* ctx, uint8_t* buf, int len); - -struct ITLSPolicy { - virtual void addref() = 0; - virtual void delref() = 0; - - // set_ca_data should import the provided certificate list and - // associate it with this policy. cert_data will point to a PEM - // encoded certificate list of trust roots. - // - // set_ca_data should return true if the operation succeeded, - // and false otherwise. After the first call to create_session for - // a given policy, set_ca_data should immediately return false - // if called. - virtual bool set_ca_data(const uint8_t* ca_data, int ca_len) = 0; - - // set_cert_data should import the provided certificate list and - // associate it with this policy. cert_data will point to a PEM - // encoded certificate list, ordered such that each certificate - // certifies the one before it. - // - // cert_data may additionally contain key information, which must - // be ignored. - // - // set_cert_data should return true if the operation succeeded, - // and false otherwise. After the first call to create_session for - // a given policy, set_cert_data should immediately return false - // if called. - virtual bool set_cert_data(const uint8_t* cert_data, int cert_len) = 0; - - // set_key_data should import the provided private key and - // associate it with this policy. key_data will point to a PEM - // encoded key, which may be encrypted. If encrypted the password - // argument should be specified, otherwise it may be NULL. - // - // key_data may additionally contain certificate information, - // which must be ignored. - // - // set_key_data should return true if the operation succeeded, and - // false otherwise. After the first call to create_session for a - // given policy, set_key_data should immediately return false if - // called. - virtual bool set_key_data(const uint8_t* key_data, int key_len, const char* password) = 0; - - // set_verify_peers should modify the validation rules for - // verifying a peer during connection handshake. The format of - // verify_peers is implementation specific. - // - // set_verify_peers should return true if the operation succeed, - // and false otherwise. After the first call to create_session for - // a given policy, set_verify_peers should immediately return - // false if called. - virtual bool set_verify_peers(int count, const uint8_t* verify_peers[], int verify_peers_len[]) = 0; - - // create_session should return a new object that implements - // ITLSSession, associated with this policy. After the first call - // to create_session for a given policy, further calls to - // ITLSPolicy::set_* will fail and return false. - // - // The newly created session should use send_func and recv_func to - // send and receive data on the underlying transport, and must - // provide send_ctx/recv_ctx to the callbacks. - // - // uid will be used to identify this session within trace events - virtual ITLSSession* create_session(bool is_client, const char *servername, TLSSendCallbackFunc send_func, void* send_ctx, TLSRecvCallbackFunc recv_func, void* recv_ctx, void* uid) = 0; -}; - -struct ITLSPlugin { - virtual void addref() = 0; - virtual void delref() = 0; - - // create_policy should return a new object that implements - // ITLSPolicy. - virtual ITLSPolicy* create_policy() = 0; - - static inline const char* get_plugin_type_name_and_version() { return "ITLSPlugin"; } -}; - -#endif /* FDB_ITLSPLUGIN_H */ diff --git a/fdbrpc/TLSConnection.actor.cpp b/fdbrpc/TLSConnection.actor.cpp deleted file mode 100644 index e412f9f81c..0000000000 --- a/fdbrpc/TLSConnection.actor.cpp +++ /dev/null @@ -1,539 +0,0 @@ -/* - * TLSConnection.actor.cpp - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors - * - * Licensed 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. - */ - -#include -#include "flow/flow.h" -#include "flow/network.h" -#include "flow/Knobs.h" -#include "fdbrpc/TLSConnection.h" -#include "fdbrpc/ITLSPlugin.h" -#include "fdbrpc/LoadPlugin.h" -#include "fdbrpc/Platform.h" -#include "fdbrpc/IAsyncFile.h" -#include "flow/actorcompiler.h" // This must be the last #include. - -// Name of specialized TLS Plugin -const char* tlsPluginName = "fdb-libressl-plugin"; - -// Must not throw an exception from this function! -static int send_func(void* ctx, const uint8_t* buf, int len) { - TLSConnection* conn = (TLSConnection*)ctx; - - try { - SendBuffer sb; - sb.bytes_sent = 0; - sb.bytes_written = len; - sb.data = buf; - sb.next = 0; - - int w = conn->conn->write( &sb ); - return w; - } catch ( Error& e ) { - TraceEvent("TLSConnectionSendError", conn->getDebugID()).suppressFor(1.0).detail("Peer", conn->getPeerAddress().toString()).error(e); - return -1; - } catch ( ... ) { - TraceEvent("TLSConnectionSendError", conn->getDebugID()).suppressFor(1.0).detail("Peer", conn->getPeerAddress()).error( unknown_error() ); - return -1; - } -} - -// Must not throw an exception from this function! -static int recv_func(void* ctx, uint8_t* buf, int len) { - TLSConnection* conn = (TLSConnection*)ctx; - - try { - int r = conn->conn->read( buf, buf + len ); - return r; - } catch ( Error& e ) { - TraceEvent("TLSConnectionRecvError", conn->getDebugID()).suppressFor(1.0).detail("Peer", conn->getPeerAddress()).error(e); - return -1; - } catch ( ... ) { - TraceEvent("TLSConnectionRecvError", conn->getDebugID()).suppressFor(1.0).detail("Peer", conn->getPeerAddress()).error( unknown_error() ); - return -1; - } -} - -ACTOR static Future handshake( TLSConnection* self ) { - state std::pair peerIP = std::make_pair(self->conn->getPeerAddress().ip, self->is_client ? self->conn->getPeerAddress().port : static_cast(0)); - if(!self->is_client) { - auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); - if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { - if (now() < iter->second.second) { - if(iter->second.first >= FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_ATTEMPTS) { - TraceEvent("TLSIncomingConnectionThrottlingWarning", self->getDebugID()).suppressFor(1.0).detail("PeerIP", peerIP.first.toString()); - wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT)); - throw connection_failed(); - } - } else { - g_network->networkInfo.serverTLSConnectionThrottler.erase(peerIP); - } - } - } - - loop { - int r = self->session->handshake(); - if(BUGGIFY_WITH_PROB(0.001)) { - r = ITLSSession::FAILED; - } - if ( r == ITLSSession::SUCCESS ) break; - if ( r == ITLSSession::FAILED ) { - TraceEvent("TLSConnectionHandshakeError", self->getDebugID()).suppressFor(1.0).detail("Peer", self->getPeerAddress()); - auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); - if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { - iter->second.first++; - } else { - g_network->networkInfo.serverTLSConnectionThrottler[peerIP] = std::make_pair(0,now() + (self->is_client ? FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_TIMEOUT : FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT)); - } - throw connection_failed(); - } - ASSERT( r == ITLSSession::WANT_WRITE || r == ITLSSession::WANT_READ ); - wait( r == ITLSSession::WANT_WRITE ? self->conn->onWritable() : self->conn->onReadable() ); - } - - TraceEvent("TLSConnectionHandshakeSuccessful", self->getDebugID()).suppressFor(1.0).detail("Peer", self->getPeerAddress()); - - return Void(); -} - -TLSConnection::TLSConnection( Reference const& conn, Reference const& policy, bool is_client, std::string host) : conn(conn), write_wants(0), read_wants(0), uid(conn->getDebugID()), is_client(is_client) { - const char * serverName = host.empty() ? NULL : host.c_str(); - session = Reference( policy->create_session(is_client, serverName, send_func, this, recv_func, this, (void*)&uid) ); - if ( !session ) { - // If session is NULL, we're trusting policy->create_session - // to have used its provided logging function to have logged - // the error - throw tls_error(); - } - handshook = handshake(this); -} - -Future TLSConnection::onWritable() { - if ( !handshook.isReady() ) - return handshook; - return - write_wants == ITLSSession::WANT_READ ? conn->onReadable() : - write_wants == ITLSSession::WANT_WRITE ? conn->onWritable() : - Void(); -} - -Future TLSConnection::onReadable() { - if ( !handshook.isReady() ) - return handshook; - return - read_wants == ITLSSession::WANT_READ ? conn->onReadable() : - read_wants == ITLSSession::WANT_WRITE ? conn->onWritable() : - Void(); -} - -int TLSConnection::read( uint8_t* begin, uint8_t* end ) { - if ( !handshook.isReady() ) return 0; - handshook.get(); - - read_wants = 0; - int r = session->read( begin, end - begin ); - if ( r > 0 ) - return r; - - if ( r == ITLSSession::FAILED ) throw connection_failed(); - - ASSERT( r == ITLSSession::WANT_WRITE || r == ITLSSession::WANT_READ ); - - read_wants = r; - return 0; -} - -int TLSConnection::write( SendBuffer const* buffer, int limit ) { - ASSERT(limit > 0); - - if ( !handshook.isReady() ) return 0; - handshook.get(); - - write_wants = 0; - int toSend = std::min(limit, buffer->bytes_written - buffer->bytes_sent); - ASSERT(toSend); - int w = session->write( buffer->data + buffer->bytes_sent, toSend ); - if ( w > 0 ) - return w; - - if ( w == ITLSSession::FAILED ) throw connection_failed(); - - ASSERT( w == ITLSSession::WANT_WRITE || w == ITLSSession::WANT_READ ); - - write_wants = w; - return 0; -} - -ACTOR Future> wrap( Reference policy, bool is_client, Future> c, std::string host) { - state Reference conn = wait(c); - try { - state Reference tlsConn(new TLSConnection( conn, policy, is_client, host )); - if(is_client) { - wait(tlsConn->handshook); - } - return tlsConn; - } catch( Error &e ) { - conn->close(); - throw e; - } -} - -Future> TLSListener::accept() { - return wrap( options->get_policy(TLSOptions::POLICY_VERIFY_PEERS), false, listener->accept(), ""); -} - -TLSNetworkConnections::TLSNetworkConnections( Reference options ) : options(options) { - network = INetworkConnections::net(); - g_network->setGlobal(INetwork::enumGlobal::enNetworkConnections, (flowGlobalType) this); -} - -ACTOR Future> waitAndFailConnection() { - wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT)); - throw connection_failed(); -} - -Future> TLSNetworkConnections::connect( NetworkAddress toAddr, std::string host) { - if ( toAddr.isTLS() ) { - NetworkAddress clearAddr( toAddr.ip, toAddr.port, toAddr.isPublic(), false ); - std::pair peerIP = std::make_pair(toAddr.ip, toAddr.port); - auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); - if(iter != g_network->networkInfo.serverTLSConnectionThrottler.end()) { - if (now() < iter->second.second) { - if(iter->second.first >= FLOW_KNOBS->TLS_CLIENT_CONNECTION_THROTTLE_ATTEMPTS) { - TraceEvent("TLSOutgoingConnectionThrottlingWarning").suppressFor(1.0).detail("PeerIP", toAddr); - return waitAndFailConnection(); - } - } else { - g_network->networkInfo.serverTLSConnectionThrottler.erase(peerIP); - } - } - - TraceEvent("TLSConnectionConnecting").suppressFor(1.0).detail("ToAddr", toAddr); - // For FDB<->FDB connections, we don't have hostnames and can't verify IP - // addresses against certificates, so we have our own peer verifying logic - // to use. For FDB<->external system connections, we can use the standard - // hostname-based certificate verification logic. - if (host.empty() || host == toAddr.ip.toString()) - return wrap(options->get_policy(TLSOptions::POLICY_VERIFY_PEERS), true, network->connect(clearAddr), std::string("")); - else - return wrap( options->get_policy(TLSOptions::POLICY_NO_VERIFY_PEERS), true, network->connect( clearAddr ), host ); - } - return network->connect( toAddr ); -} - -Future> TLSNetworkConnections::resolveTCPEndpoint( std::string host, std::string service) { - return network->resolveTCPEndpoint( host, service ); -} - -Reference TLSNetworkConnections::listen( NetworkAddress localAddr ) { - if ( localAddr.isTLS() ) { - NetworkAddress clearAddr( localAddr.ip, localAddr.port, localAddr.isPublic(), false ); - TraceEvent("TLSConnectionListening").detail("OnAddr", localAddr); - return Reference(new TLSListener( options, network->listen( clearAddr ) )); - } - return network->listen( localAddr ); -} - -// 5MB for loading files into memory -#define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) - -void TLSOptions::set_cert_file( std::string const& cert_file ) { - try { - TraceEvent("TLSConnectionSettingCertFile").detail("CertFilePath", cert_file); - policyInfo.cert_path = cert_file; - set_cert_data( readFileBytes( cert_file, CERT_FILE_MAX_SIZE ) ); - } catch ( Error& e) { - TraceEvent(SevError, "TLSOptionsSetCertFileError").detail("Filename", cert_file).error(e).GetLastError(); - throw; - } -} - -void TLSOptions::set_ca_file(std::string const& ca_file) { - try { - TraceEvent("TLSConnectionSettingCAFile").detail("CAPath", ca_file); - policyInfo.ca_path = ca_file; - set_ca_data(readFileBytes(ca_file, CERT_FILE_MAX_SIZE)); - } - catch (Error& e) { - TraceEvent(SevError, "TLSOptionsSetCertAError").detail("Filename", ca_file).error(e).GetLastError(); - throw; - } -} - -void TLSOptions::set_ca_data(std::string const& ca_data) { - if (!policyVerifyPeersSet.get() || !policyVerifyPeersNotSet.get()) - init_plugin(); - - TraceEvent("TLSConnectionSettingCAData").detail("CADataSize", ca_data.size()); - policyInfo.ca_contents = Standalone(ca_data); - if (!policyVerifyPeersSet.get()->set_ca_data((const uint8_t*)&ca_data[0], ca_data.size())) - throw tls_error(); - if (!policyVerifyPeersNotSet.get()->set_ca_data((const uint8_t*)&ca_data[0], ca_data.size())) - throw tls_error(); - - ca_set = true; -} - -void TLSOptions::set_cert_data( std::string const& cert_data ) { - if (!policyVerifyPeersSet.get() || !policyVerifyPeersNotSet.get()) - init_plugin(); - - TraceEvent("TLSConnectionSettingCertData").detail("CertDataSize", cert_data.size()); - policyInfo.cert_contents = Standalone(cert_data); - if ( !policyVerifyPeersSet.get()->set_cert_data( (const uint8_t*)&cert_data[0], cert_data.size() ) ) - throw tls_error(); - if (!policyVerifyPeersNotSet.get()->set_cert_data((const uint8_t*)&cert_data[0], cert_data.size())) - throw tls_error(); - - certs_set = true; -} - -void TLSOptions::set_key_password(std::string const& password) { - TraceEvent("TLSConnectionSettingPassword"); - policyInfo.keyPassword = password; -} - -void TLSOptions::set_key_file( std::string const& key_file ) { - try { - TraceEvent("TLSConnectionSettingKeyFile").detail("KeyFilePath", key_file); - policyInfo.key_path = key_file; - set_key_data( readFileBytes( key_file, CERT_FILE_MAX_SIZE ) ); - } catch ( Error& e) { - TraceEvent(SevError, "TLSOptionsSetKeyFileError").detail("Filename", key_file).error(e).GetLastError(); - throw; - } -} - -void TLSOptions::set_key_data( std::string const& key_data ) { - if (!policyVerifyPeersSet.get() || !policyVerifyPeersNotSet.get()) - init_plugin(); - const char *passphrase = policyInfo.keyPassword.empty() ? NULL : policyInfo.keyPassword.c_str(); - TraceEvent("TLSConnectionSettingKeyData").detail("KeyDataSize", key_data.size()); - policyInfo.key_contents = Standalone(key_data); - if ( !policyVerifyPeersSet.get()->set_key_data( (const uint8_t*)&key_data[0], key_data.size(), passphrase) ) - throw tls_error(); - if (!policyVerifyPeersNotSet.get()->set_key_data((const uint8_t*)&key_data[0], key_data.size(), passphrase)) - throw tls_error(); - - key_set = true; -} - -void TLSOptions::set_verify_peers( std::vector const& verify_peers ) { - if (!policyVerifyPeersSet.get()) - init_plugin(); - { - TraceEvent e("TLSConnectionSettingVerifyPeers"); - for (int i = 0; i < verify_peers.size(); i++) - e.detail(std::string("Value" + std::to_string(i)).c_str(), verify_peers[i].c_str()); - } - std::unique_ptr verify_peers_arr(new const uint8_t*[verify_peers.size()]); - std::unique_ptr verify_peers_len(new int[verify_peers.size()]); - for (int i = 0; i < verify_peers.size(); i++) { - verify_peers_arr[i] = (const uint8_t *)&verify_peers[i][0]; - verify_peers_len[i] = verify_peers[i].size(); - } - - if (!policyVerifyPeersSet.get()->set_verify_peers(verify_peers.size(), verify_peers_arr.get(), verify_peers_len.get())) - throw tls_error(); - - policyInfo.verify_peers = verify_peers; - verify_peers_set = true; -} - -ACTOR static Future>> readEntireFile( std::string filename ) { - state Reference file = wait(IAsyncFileSystem::filesystem()->open(filename, IAsyncFile::OPEN_READONLY | IAsyncFile::OPEN_UNCACHED, 0)); - state int64_t filesize = wait(file->size()); - state Standalone buf = makeString(filesize); - int rc = wait(file->read(mutateString(buf), filesize, 0)); - if (rc != filesize) { - // File modified during read, probably. The mtime should change, and thus we'll be called again. - return tls_error(); - } - return buf; -} - -ACTOR static Future watchFileForChanges( std::string filename, AsyncVar> *contents_var ) { - state std::time_t lastModTime = wait(IAsyncFileSystem::filesystem()->lastWriteTime(filename)); - loop { - wait(delay(FLOW_KNOBS->TLS_CERT_REFRESH_DELAY_SECONDS)); - std::time_t modtime = wait(IAsyncFileSystem::filesystem()->lastWriteTime(filename)); - if (lastModTime != modtime) { - lastModTime = modtime; - ErrorOr> contents = wait(readEntireFile(filename)); - if (contents.present()) { - contents_var->set(contents.get()); - } - } - } -} - -ACTOR static Future reloadConfigurationOnChange( TLSOptions::PolicyInfo *pci, Reference plugin, AsyncVar> *realVerifyPeersPolicy, AsyncVar> *realNoVerifyPeersPolicy ) { - if (FLOW_KNOBS->TLS_CERT_REFRESH_DELAY_SECONDS <= 0) { - return Void(); - } - loop { - // Early in bootup, the filesystem might not be initialized yet. Wait until it is. - if (IAsyncFileSystem::filesystem() != nullptr) { - break; - } - wait(delay(1.0)); - } - state int mismatches = 0; - state AsyncVar> ca_var; - state AsyncVar> key_var; - state AsyncVar> cert_var; - state std::vector> lifetimes; - if (!pci->ca_path.empty()) lifetimes.push_back(watchFileForChanges(pci->ca_path, &ca_var)); - if (!pci->key_path.empty()) lifetimes.push_back(watchFileForChanges(pci->key_path, &key_var)); - if (!pci->cert_path.empty()) lifetimes.push_back(watchFileForChanges(pci->cert_path, &cert_var)); - loop { - state Future ca_changed = ca_var.onChange(); - state Future key_changed = key_var.onChange(); - state Future cert_changed = cert_var.onChange(); - wait( ca_changed || key_changed || cert_changed ); - if (ca_changed.isReady()) { - TraceEvent(SevInfo, "TLSRefreshCAChanged").detail("path", pci->ca_path).detail("length", ca_var.get().size()); - pci->ca_contents = ca_var.get(); - } - if (key_changed.isReady()) { - TraceEvent(SevInfo, "TLSRefreshKeyChanged").detail("path", pci->key_path).detail("length", key_var.get().size()); - pci->key_contents = key_var.get(); - } - if (cert_changed.isReady()) { - TraceEvent(SevInfo, "TLSRefreshCertChanged").detail("path", pci->cert_path).detail("length", cert_var.get().size()); - pci->cert_contents = cert_var.get(); - } - bool rc = true; - Reference verifypeers = Reference(plugin->create_policy()); - Reference noverifypeers = Reference(plugin->create_policy()); - loop { - // Don't actually loop. We're just using loop/break as a `goto err`. - // This loop always ends with an unconditional break. - rc = verifypeers->set_ca_data(pci->ca_contents.begin(), pci->ca_contents.size()); - if (!rc) break; - rc = verifypeers->set_key_data(pci->key_contents.begin(), pci->key_contents.size(), pci->keyPassword.c_str()); - if (!rc) break; - rc = verifypeers->set_cert_data(pci->cert_contents.begin(), pci->cert_contents.size()); - if (!rc) break; - { - std::unique_ptr verify_peers_arr(new const uint8_t*[pci->verify_peers.size()]); - std::unique_ptr verify_peers_len(new int[pci->verify_peers.size()]); - for (int i = 0; i < pci->verify_peers.size(); i++) { - verify_peers_arr[i] = (const uint8_t *)&pci->verify_peers[i][0]; - verify_peers_len[i] = pci->verify_peers[i].size(); - } - rc = verifypeers->set_verify_peers(pci->verify_peers.size(), verify_peers_arr.get(), verify_peers_len.get()); - if (!rc) break; - } - rc = noverifypeers->set_ca_data(pci->ca_contents.begin(), pci->ca_contents.size()); - if (!rc) break; - rc = noverifypeers->set_key_data(pci->key_contents.begin(), pci->key_contents.size(), pci->keyPassword.c_str()); - if (!rc) break; - rc = noverifypeers->set_cert_data(pci->cert_contents.begin(), pci->cert_contents.size()); - if (!rc) break; - break; - } - - if (rc) { - TraceEvent(SevInfo, "TLSCertificateRefreshSucceeded"); - realVerifyPeersPolicy->set(verifypeers); - realNoVerifyPeersPolicy->set(noverifypeers); - mismatches = 0; - } else { - // Some files didn't match up, they should in the future, and we'll retry then. - mismatches++; - TraceEvent(SevWarn, "TLSCertificateRefreshMismatch").detail("mismatches", mismatches); - } - } -} - -const char *defaultCertFileName = "fdb.pem"; - -Reference TLSOptions::get_policy(PolicyType type) { - if ( !certs_set ) { - if ( !platform::getEnvironmentVar( "FDB_TLS_CERTIFICATE_FILE", policyInfo.cert_path ) ) - policyInfo.cert_path = fileExists(defaultCertFileName) ? defaultCertFileName : joinPath(platform::getDefaultConfigPath(), defaultCertFileName); - set_cert_file( policyInfo.cert_path ); - } - if ( !key_set ) { - if ( policyInfo.keyPassword.empty() ) - platform::getEnvironmentVar( "FDB_TLS_PASSWORD", policyInfo.keyPassword ); - if ( !platform::getEnvironmentVar( "FDB_TLS_KEY_FILE", policyInfo.key_path ) ) - policyInfo.key_path = fileExists(defaultCertFileName) ? defaultCertFileName : joinPath(platform::getDefaultConfigPath(), defaultCertFileName); - set_key_file( policyInfo.key_path ); - } - if( !verify_peers_set ) { - std::string verify_peers; - if (platform::getEnvironmentVar("FDB_TLS_VERIFY_PEERS", verify_peers)) - set_verify_peers({ verify_peers }); - else - set_verify_peers({ std::string("Check.Valid=1")}); - } - if (!ca_set) { - if (platform::getEnvironmentVar("FDB_TLS_CA_FILE", policyInfo.ca_path)) - set_ca_file(policyInfo.ca_path); - } - - if (!configurationReloader.present()) { - configurationReloader = reloadConfigurationOnChange(&policyInfo, plugin, &policyVerifyPeersSet, &policyVerifyPeersNotSet); - } - - Reference policy; - switch (type) { - case POLICY_VERIFY_PEERS: - policy = policyVerifyPeersSet.get(); - break; - case POLICY_NO_VERIFY_PEERS: - policy = policyVerifyPeersNotSet.get(); - break; - default: - ASSERT_ABORT(0); - } - return policy; -} - -void TLSOptions::init_plugin() { - - TraceEvent("TLSConnectionLoadingPlugin").detail("Plugin", tlsPluginName); - - plugin = loadPlugin( tlsPluginName ); - - if ( !plugin ) { - TraceEvent(SevError, "TLSConnectionPluginInitError").detail("Plugin", tlsPluginName).GetLastError(); - throw tls_error(); - } - - policyVerifyPeersSet = AsyncVar>(Reference(plugin->create_policy())); - if ( !policyVerifyPeersSet.get()) { - // Hopefully create_policy logged something with the log func - TraceEvent(SevError, "TLSConnectionCreatePolicyVerifyPeersSetError"); - throw tls_error(); - } - - policyVerifyPeersNotSet = AsyncVar>(Reference(plugin->create_policy())); - if (!policyVerifyPeersNotSet.get()) { - // Hopefully create_policy logged something with the log func - TraceEvent(SevError, "TLSConnectionCreatePolicyVerifyPeersNotSetError"); - throw tls_error(); - } -} - -bool TLSOptions::enabled() { - return policyVerifyPeersSet.get().isValid() && policyVerifyPeersNotSet.get().isValid(); -} diff --git a/fdbrpc/TLSConnection.h b/fdbrpc/TLSConnection.h deleted file mode 100644 index b32b6f5f0b..0000000000 --- a/fdbrpc/TLSConnection.h +++ /dev/null @@ -1,175 +0,0 @@ -/* - * TLSConnection.h - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors - * - * Licensed 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. - */ - -#ifndef FLOW_TLSCONNECTION_H -#define FLOW_TLSCONNECTION_H - -#pragma once - -#include "flow/Platform.h" - -#include "fdbrpc/ITLSPlugin.h" - -struct TLSConnection : IConnection, ReferenceCounted { - Reference conn; - Reference session; - - Future handshook; - - int write_wants, read_wants; - - UID uid; - bool is_client; - - virtual void addref() { ReferenceCounted::addref(); } - virtual void delref() { ReferenceCounted::delref(); } - - TLSConnection( Reference const& conn, Reference const& policy, bool is_client, std::string host); - ~TLSConnection() { - // Here for ordering to make sure we delref the ITLSSession - // which has a pointer to this object - session.clear(); - } - - virtual void close() { conn->close(); } - - virtual Future acceptHandshake() { return Void(); } - virtual Future connectHandshake() { return Void(); } - - virtual Future onWritable(); - - virtual Future onReadable(); - - virtual int read( uint8_t* begin, uint8_t* end ); - - virtual int write( SendBuffer const* buffer, int limit); - - virtual NetworkAddress getPeerAddress() { - NetworkAddress a = conn->getPeerAddress(); - return NetworkAddress(a.ip, a.port, a.isPublic(), true); - } - - virtual UID getDebugID() { return uid; } -}; - -struct TLSOptions : ReferenceCounted { - enum { OPT_TLS = 100000, OPT_TLS_PLUGIN, OPT_TLS_CERTIFICATES, OPT_TLS_KEY, OPT_TLS_VERIFY_PEERS, OPT_TLS_CA_FILE, OPT_TLS_PASSWORD }; - enum PolicyType { POLICY_VERIFY_PEERS = 1, POLICY_NO_VERIFY_PEERS }; - TLSOptions() : certs_set(false), key_set(false), verify_peers_set(false), ca_set(false) { -#ifndef TLS_DISABLED - init_plugin( ); -#endif - } - - void set_cert_file( std::string const& cert_file ); - void set_cert_data( std::string const& cert_data ); - void set_ca_file(std::string const& ca_file); - void set_ca_data(std::string const& ca_data); - // If there is a passphrase, this api should be called prior to setting key for the passphrase to be used - void set_key_password( std::string const& password ); - void set_key_file( std::string const& key_file ); - void set_key_data( std::string const& key_data ); - void set_verify_peers( std::vector const& verify_peers ); - - Reference get_policy(PolicyType type); - bool enabled(); - - struct PolicyInfo { - std::string ca_path; - Standalone ca_contents; - std::string key_path; - std::string keyPassword; - Standalone key_contents; - std::string cert_path; - Standalone cert_contents; - std::vector verify_peers; - }; - -private: - void init_plugin(); - - Reference plugin; - PolicyInfo policyInfo; - AsyncVar> policyVerifyPeersSet; - AsyncVar> policyVerifyPeersNotSet; - Optional> configurationReloader; - - bool certs_set, key_set, verify_peers_set, ca_set; -}; - -struct TLSListener : IListener, ReferenceCounted { - Reference listener; - Reference options; - - TLSListener( Reference options, Reference listener ) : options(options), listener(listener) {} - - virtual void addref() { ReferenceCounted::addref(); } - virtual void delref() { ReferenceCounted::delref(); } - - virtual Future> accept(); - - virtual NetworkAddress getListenAddress() { return listener->getListenAddress(); } -}; - -struct TLSNetworkConnections : INetworkConnections { - INetworkConnections *network; - - explicit TLSNetworkConnections( Reference options ); - - virtual Future> connect( NetworkAddress toAddr, std::string host ); - virtual Future> resolveTCPEndpoint( std::string host, std::string service); - - virtual Reference listen( NetworkAddress localAddr ); - -private: - Reference options; -}; - -#define TLS_PLUGIN_FLAG "--tls_plugin" -#define TLS_CERTIFICATE_FILE_FLAG "--tls_certificate_file" -#define TLS_KEY_FILE_FLAG "--tls_key_file" -#define TLS_VERIFY_PEERS_FLAG "--tls_verify_peers" -#define TLS_CA_FILE_FLAG "--tls_ca_file" -#define TLS_PASSWORD_FLAG "--tls_password" - -#define TLS_OPTION_FLAGS \ - { TLSOptions::OPT_TLS_PLUGIN, TLS_PLUGIN_FLAG, SO_REQ_SEP }, \ - { TLSOptions::OPT_TLS_CERTIFICATES, TLS_CERTIFICATE_FILE_FLAG, SO_REQ_SEP }, \ - { TLSOptions::OPT_TLS_KEY, TLS_KEY_FILE_FLAG, SO_REQ_SEP }, \ - { TLSOptions::OPT_TLS_VERIFY_PEERS, TLS_VERIFY_PEERS_FLAG, SO_REQ_SEP }, \ - { TLSOptions::OPT_TLS_PASSWORD, TLS_PASSWORD_FLAG, SO_REQ_SEP }, \ - { TLSOptions::OPT_TLS_CA_FILE, TLS_CA_FILE_FLAG, SO_REQ_SEP }, - -#define TLS_HELP \ - " " TLS_CERTIFICATE_FILE_FLAG " CERTFILE\n" \ - " The path of a file containing the TLS certificate and CA\n" \ - " chain.\n" \ - " " TLS_CA_FILE_FLAG " CERTAUTHFILE\n" \ - " The path of a file containing the CA certificates chain.\n" \ - " " TLS_KEY_FILE_FLAG " KEYFILE\n" \ - " The path of a file containing the private key corresponding\n" \ - " to the TLS certificate.\n" \ - " " TLS_PASSWORD_FLAG " PASSCODE\n" \ - " The passphrase of encrypted private key\n" \ - " " TLS_VERIFY_PEERS_FLAG " CONSTRAINTS\n" \ - " The constraints by which to validate TLS peers. The contents\n" \ - " and format of CONSTRAINTS are plugin-specific.\n" - -#endif /* FLOW_TLSCONNECTION_H */ diff --git a/fdbrpc/fdbrpc.vcxproj b/fdbrpc/fdbrpc.vcxproj index b77c8d24f8..e71d4b41b0 100644 --- a/fdbrpc/fdbrpc.vcxproj +++ b/fdbrpc/fdbrpc.vcxproj @@ -35,7 +35,6 @@ - @@ -91,7 +90,6 @@ - @@ -110,7 +108,6 @@ - diff --git a/fdbrpc/fdbrpc.vcxproj.filters b/fdbrpc/fdbrpc.vcxproj.filters index 0aed89c29f..b9f7bf63e3 100644 --- a/fdbrpc/fdbrpc.vcxproj.filters +++ b/fdbrpc/fdbrpc.vcxproj.filters @@ -10,7 +10,6 @@ - @@ -129,7 +128,6 @@ zlib - @@ -144,7 +142,6 @@ - diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 6f5d479494..918f0510bb 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -1590,10 +1590,10 @@ public: machines.erase(machineId); } - Sim2() : time(0.0), taskCount(0), yielded(false), yield_limit(0), currentTaskID(TaskPriority::Zero), sslContext(boost::asio::ssl::context(boost::asio::ssl::context::tlsv12)) { + Sim2() : time(0.0), taskCount(0), yielded(false), yield_limit(0), currentTaskID(TaskPriority::Zero) { // Not letting currentProcess be NULL eliminates some annoying special cases currentProcess = new ProcessInfo("NoMachine", LocalityData(Optional>(), StringRef(), StringRef(), StringRef()), ProcessClass(), {NetworkAddress()}, this, "", ""); - g_network = net2 = newNet2(&sslContext, false, true); + g_network = net2 = newNet2(false, true); Net2FileSystem::newFileSystem(); check_yield(TaskPriority::Zero); } @@ -1692,7 +1692,6 @@ public: //Sim2Net network; INetwork *net2; - boost::asio::ssl::context sslContext; //Map from machine IP -> machine disk space info std::map diskSpaceMap; diff --git a/fdbserver/SimulatedCluster.h b/fdbserver/SimulatedCluster.h index d7ed3ed1ab..bc2d4fd510 100644 --- a/fdbserver/SimulatedCluster.h +++ b/fdbserver/SimulatedCluster.h @@ -18,8 +18,6 @@ * limitations under the License. */ -#include "fdbrpc/TLSConnection.h" - #ifndef FDBSERVER_SIMULATEDCLUSTER_H #define FDBSERVER_SIMULATEDCLUSTER_H #pragma once diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index 8c4979057c..233eaba14b 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -52,7 +52,6 @@ #include "fdbserver/workloads/workloads.actor.h" #include #include "fdbserver/Status.h" -#include "fdbrpc/TLSConnection.h" #include "fdbrpc/Net2FileSystem.h" #include "fdbrpc/Platform.h" #include "fdbrpc/AsyncFileCached.actor.h" @@ -964,7 +963,6 @@ int main(int argc, char* argv[]) { int minTesterCount = 1; bool testOnServers = false; - boost::asio::ssl::context sslContext(boost::asio::ssl::context::tlsv12); Reference tlsPolicy = Reference(new TLSPolicy(TLSPolicy::Is::SERVER)); TLSParams tlsParams; std::vector tlsVerifyPeers; @@ -1335,22 +1333,22 @@ int main(int argc, char* argv[]) { whitelistBinPaths = args.OptionArg(); break; #ifndef TLS_DISABLED - case TLSOptions::OPT_TLS_PLUGIN: + case TLSParams::OPT_TLS_PLUGIN: args.OptionArg(); break; - case TLSOptions::OPT_TLS_CERTIFICATES: + case TLSParams::OPT_TLS_CERTIFICATES: tlsParams.tlsCertPath = args.OptionArg(); break; - case TLSOptions::OPT_TLS_PASSWORD: + case TLSParams::OPT_TLS_PASSWORD: tlsParams.tlsPassword = args.OptionArg(); break; - case TLSOptions::OPT_TLS_CA_FILE: + case TLSParams::OPT_TLS_CA_FILE: tlsParams.tlsCAPath = args.OptionArg(); break; - case TLSOptions::OPT_TLS_KEY: + case TLSParams::OPT_TLS_KEY: tlsParams.tlsKeyPath = args.OptionArg(); break; - case TLSOptions::OPT_TLS_VERIFY_PEERS: + case TLSParams::OPT_TLS_VERIFY_PEERS: tlsVerifyPeers.push_back(args.OptionArg()); break; #endif @@ -1560,7 +1558,7 @@ int main(int argc, char* argv[]) { tlsPolicy->set_verify_peers( tlsVerifyPeers ); } #endif - g_network = newNet2(&sslContext, useThreadPool, true, tlsPolicy, tlsParams); + g_network = newNet2(useThreadPool, true, tlsPolicy, tlsParams); FlowTransport::createInstance(false, 1); const bool expectsPublicAddress = (role == FDBD || role == NetworkTestServer || role == Restore); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 8547f74e29..bc7b6cc768 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -113,7 +113,7 @@ thread_local INetwork* thread_network = 0; class Net2 sealed : public INetwork, public INetworkConnections { public: - Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, const TLSParams& tlsParams); + Net2(bool useThreadPool, bool useMetrics, Reference policy, const TLSParams& tlsParams); void run(); void initMetrics(); @@ -156,7 +156,7 @@ public: //private: ASIOReactor reactor; - boost::asio::ssl::context* sslContext; + boost::asio::ssl::context sslContext; std::string tlsPassword; std::string get_password() const { @@ -832,7 +832,11 @@ struct PromiseTask : public Task, public FastAllocated { // TODO: Move to a headerfile and delete all the copies of this. #define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) -Net2::Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslContext, const TLSParams& tlsParams) +bool insecurely_always_accept(bool _1, boost::asio::ssl::verify_context& _2) { + return true; +} + +Net2::Net2(bool useThreadPool, bool useMetrics, Reference policy, const TLSParams& tlsParams) : useThreadPool(useThreadPool), network(this), reactor(this), @@ -842,33 +846,42 @@ Net2::Net2(bool useThreadPool, bool useMetrics, boost::asio::ssl::context* sslCo tsc_begin(0), tsc_end(0), taskBegin(0), currentTaskID(TaskPriority::DefaultYield), lastMinTaskID(TaskPriority::Zero), numYields(0), - sslContext(sslContext), - tlsPassword(tlsParams.tlsPassword) + tlsPassword(tlsParams.tlsPassword), + sslContext(boost::asio::ssl::context(boost::asio::ssl::context::tlsv12)) + { TraceEvent("Net2Starting"); - if(sslContext) { - sslContext->set_password_callback(std::bind(&Net2::get_password, this)); + sslContext.set_options(boost::asio::ssl::context::default_workarounds); + sslContext.set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); + if (policy) { + sslContext.set_verify_callback([policy](bool preverified, boost::asio::ssl::verify_context& ctx) { + return policy->verify_peer(preverified, ctx.native_handle()); + }); + } else { + sslContext.set_verify_callback(boost::bind(&insecurely_always_accept, _1, _2)); + } - if (tlsParams.tlsCertPath.size() ) { - sslContext->use_certificate_chain_file(tlsParams.tlsCertPath); - } - if (tlsParams.tlsCertBytes.size() ) { - sslContext->use_certificate(boost::asio::buffer(tlsParams.tlsCertBytes.data(), tlsParams.tlsCertBytes.size()), boost::asio::ssl::context::pem); - } - if (tlsParams.tlsCAPath.size()) { - std::string cert = readFileBytes(tlsParams.tlsCAPath, CERT_FILE_MAX_SIZE); - sslContext->add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); - } - if (tlsParams.tlsCABytes.size()) { - sslContext->add_certificate_authority(boost::asio::buffer(tlsParams.tlsCABytes.data(), tlsParams.tlsCABytes.size())); - } - if (tlsParams.tlsKeyPath.size()) { - sslContext->use_private_key_file(tlsParams.tlsKeyPath, boost::asio::ssl::context::pem); - } - if (tlsParams.tlsKeyBytes.size()) { - sslContext->use_private_key(boost::asio::buffer(tlsParams.tlsKeyBytes.data(), tlsParams.tlsKeyBytes.size()), boost::asio::ssl::context::pem); - } + sslContext.set_password_callback(std::bind(&Net2::get_password, this)); + + if (tlsParams.tlsCertPath.size() ) { + sslContext.use_certificate_chain_file(tlsParams.tlsCertPath); + } + if (tlsParams.tlsCertBytes.size() ) { + sslContext.use_certificate(boost::asio::buffer(tlsParams.tlsCertBytes.data(), tlsParams.tlsCertBytes.size()), boost::asio::ssl::context::pem); + } + if (tlsParams.tlsCAPath.size()) { + std::string cert = readFileBytes(tlsParams.tlsCAPath, CERT_FILE_MAX_SIZE); + sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); + } + if (tlsParams.tlsCABytes.size()) { + sslContext.add_certificate_authority(boost::asio::buffer(tlsParams.tlsCABytes.data(), tlsParams.tlsCABytes.size())); + } + if (tlsParams.tlsKeyPath.size()) { + sslContext.use_private_key_file(tlsParams.tlsKeyPath, boost::asio::ssl::context::pem); + } + if (tlsParams.tlsKeyBytes.size()) { + sslContext.use_private_key(boost::asio::buffer(tlsParams.tlsKeyBytes.data(), tlsParams.tlsKeyBytes.size()), boost::asio::ssl::context::pem); } // Set the global members @@ -1248,7 +1261,7 @@ THREAD_HANDLE Net2::startThread( THREAD_FUNC_RETURN (*func) (void*), void *arg ) Future< Reference > Net2::connect( NetworkAddress toAddr, std::string host ) { if ( toAddr.isTLS() ) { - return SSLConnection::connect(&this->reactor.ios, this->sslContext, toAddr); + return SSLConnection::connect(&this->reactor.ios, &this->sslContext, toAddr); } return Connection::connect(&this->reactor.ios, toAddr); @@ -1325,7 +1338,7 @@ bool Net2::isAddressOnThisHost( NetworkAddress const& addr ) { Reference Net2::listen( NetworkAddress localAddr ) { try { if ( localAddr.isTLS() ) { - return Reference(new SSLListener( reactor.ios, this->sslContext, localAddr )); + return Reference(new SSLListener( reactor.ios, &this->sslContext, localAddr )); } return Reference( new Listener( reactor.ios, localAddr ) ); } catch (boost::system::system_error const& e) { @@ -1421,22 +1434,9 @@ void ASIOReactor::wake() { } // namespace net2 -bool insecurely_always_accept(bool _1, boost::asio::ssl::verify_context& _2) { - return true; -} - -INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool, bool useMetrics, Reference policy, const TLSParams& tlsParams) { +INetwork* newNet2(bool useThreadPool, bool useMetrics, Reference policy, const TLSParams& tlsParams) { try { - sslContext->set_options(boost::asio::ssl::context::default_workarounds); - sslContext->set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); - if (policy) { - sslContext->set_verify_callback([policy](bool preverified, boost::asio::ssl::verify_context& ctx) { - return policy->verify_peer(preverified, ctx.native_handle()); - }); - } else { - sslContext->set_verify_callback(boost::bind(&insecurely_always_accept, _1, _2)); - } - N2::g_net2 = new N2::Net2(useThreadPool, useMetrics, sslContext, tlsParams); + N2::g_net2 = new N2::Net2(useThreadPool, useMetrics, policy, tlsParams); } catch(boost::system::system_error e) { TraceEvent("Net2InitError").detail("Message", e.what()); diff --git a/flow/TLSPolicy.h b/flow/TLSPolicy.h index e440176c30..a50fa879e3 100644 --- a/flow/TLSPolicy.h +++ b/flow/TLSPolicy.h @@ -30,6 +30,8 @@ #include "flow/FastRef.h" struct TLSParams { + enum { OPT_TLS = 100000, OPT_TLS_PLUGIN, OPT_TLS_CERTIFICATES, OPT_TLS_KEY, OPT_TLS_VERIFY_PEERS, OPT_TLS_CA_FILE, OPT_TLS_PASSWORD }; + std::string tlsCertPath, tlsKeyPath, tlsCAPath, tlsPassword; std::string tlsCertBytes, tlsKeyBytes, tlsCABytes; }; @@ -105,4 +107,34 @@ public: bool is_client; }; +#define TLS_PLUGIN_FLAG "--tls_plugin" +#define TLS_CERTIFICATE_FILE_FLAG "--tls_certificate_file" +#define TLS_KEY_FILE_FLAG "--tls_key_file" +#define TLS_VERIFY_PEERS_FLAG "--tls_verify_peers" +#define TLS_CA_FILE_FLAG "--tls_ca_file" +#define TLS_PASSWORD_FLAG "--tls_password" + +#define TLS_OPTION_FLAGS \ + { TLSParams::OPT_TLS_PLUGIN, TLS_PLUGIN_FLAG, SO_REQ_SEP }, \ + { TLSParams::OPT_TLS_CERTIFICATES, TLS_CERTIFICATE_FILE_FLAG, SO_REQ_SEP }, \ + { TLSParams::OPT_TLS_KEY, TLS_KEY_FILE_FLAG, SO_REQ_SEP }, \ + { TLSParams::OPT_TLS_VERIFY_PEERS, TLS_VERIFY_PEERS_FLAG, SO_REQ_SEP }, \ + { TLSParams::OPT_TLS_PASSWORD, TLS_PASSWORD_FLAG, SO_REQ_SEP }, \ + { TLSParams::OPT_TLS_CA_FILE, TLS_CA_FILE_FLAG, SO_REQ_SEP }, + +#define TLS_HELP \ + " " TLS_CERTIFICATE_FILE_FLAG " CERTFILE\n" \ + " The path of a file containing the TLS certificate and CA\n" \ + " chain.\n" \ + " " TLS_CA_FILE_FLAG " CERTAUTHFILE\n" \ + " The path of a file containing the CA certificates chain.\n" \ + " " TLS_KEY_FILE_FLAG " KEYFILE\n" \ + " The path of a file containing the private key corresponding\n" \ + " to the TLS certificate.\n" \ + " " TLS_PASSWORD_FLAG " PASSCODE\n" \ + " The passphrase of encrypted private key\n" \ + " " TLS_VERIFY_PEERS_FLAG " CONSTRAINTS\n" \ + " The constraints by which to validate TLS peers. The contents\n" \ + " and format of CONSTRAINTS are plugin-specific.\n" + #endif diff --git a/flow/network.h b/flow/network.h index 8df670307c..a9353e6d0f 100644 --- a/flow/network.h +++ b/flow/network.h @@ -406,7 +406,7 @@ typedef NetworkAddressList (*NetworkAddressesFuncPtr)(); class INetwork; extern INetwork* g_network; -extern INetwork* newNet2(boost::asio::ssl::context* sslContext, bool useThreadPool = false, bool useMetrics = false, Reference policy = Reference(), const TLSParams& tlsParams = TLSParams()); +extern INetwork* newNet2(bool useThreadPool = false, bool useMetrics = false, Reference policy = Reference(), const TLSParams& tlsParams = TLSParams()); class INetwork { public: From fd8a58b03540851e670323ae278a04c609325494 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 19 Feb 2020 18:37:47 -0800 Subject: [PATCH 26/44] re-added support for the TLS_DISABLED flag --- fdbclient/NativeAPI.actor.cpp | 4 ++ flow/Net2.actor.cpp | 112 ++++++++++++++++++++-------------- flow/TLSPolicy.cpp | 9 ++- flow/TLSPolicy.h | 19 +++--- flow/network.h | 2 + 5 files changed, 90 insertions(+), 56 deletions(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index a0863ce0db..9b1aaec603 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -73,9 +73,11 @@ TLSParams tlsParams; static Reference tlsPolicy; static void initTLSPolicy() { +#ifndef TLS_DISABLED if (!tlsPolicy) { tlsPolicy = Reference(new TLSPolicy(TLSPolicy::Is::CLIENT)); } +#endif } static const Key CLIENT_LATENCY_INFO_PREFIX = LiteralStringRef("client_latency/"); @@ -923,11 +925,13 @@ void setNetworkOption(FDBNetworkOptions::Option option, Optional valu case FDBNetworkOptions::TLS_VERIFY_PEERS: validateOptionValue(value, true); initTLSPolicy(); +#ifndef TLS_DISABLED if (!tlsPolicy->set_verify_peers({ value.get().toString() })) { TraceEvent(SevWarnAlways, "TLSValidationSetError") .detail("Input", value.get().toString() ); throw invalid_option_value(); } +#endif break; case FDBNetworkOptions::CLIENT_BUGGIFY_ENABLE: enableBuggify(true, BuggifyType::Client); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index bc7b6cc768..4fdc769b76 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -50,8 +50,6 @@ intptr_t g_stackYieldLimit = 0; using namespace boost::asio::ip; -typedef boost::asio::ssl::stream ssl_socket; - #if defined(__linux__) #include @@ -156,7 +154,9 @@ public: //private: ASIOReactor reactor; +#ifndef TLS_DISABLED boost::asio::ssl::context sslContext; +#endif std::string tlsPassword; std::string get_password() const { @@ -253,7 +253,11 @@ public: try { if (error) { // Log the error... - TraceEvent(SevWarn, errContext, errID).suppressFor(1.0).detail("Message", error.value()).detail("WhichMeans", TLSPolicy::ErrorString(error)); + TraceEvent(SevWarn, errContext, errID).suppressFor(1.0).detail("Message", error.value()) +#ifndef TLS_DISABLED + .detail("WhichMeans", TLSPolicy::ErrorString(error)) +#endif + ; p.sendError( connection_failed() ); } else p.send( Void() ); @@ -442,6 +446,51 @@ private: } }; +class Listener : public IListener, ReferenceCounted { + NetworkAddress listenAddress; + tcp::acceptor acceptor; + +public: + Listener( boost::asio::io_service& io_service, NetworkAddress listenAddress ) + : listenAddress(listenAddress), acceptor( io_service, tcpEndpoint( listenAddress ) ) + { + platform::setCloseOnExec(acceptor.native_handle()); + } + + virtual void addref() { ReferenceCounted::addref(); } + virtual void delref() { ReferenceCounted::delref(); } + + // Returns one incoming connection when it is available + virtual Future> accept() { + return doAccept( this ); + } + + virtual NetworkAddress getListenAddress() { return listenAddress; } + +private: + ACTOR static Future> doAccept( Listener* self ) { + state Reference conn( new Connection( self->acceptor.get_io_service() ) ); + state tcp::acceptor::endpoint_type peer_endpoint; + try { + BindPromise p("N2_AcceptError", UID()); + auto f = p.getFuture(); + self->acceptor.async_accept( conn->getSocket(), peer_endpoint, std::move(p) ); + wait( f ); + auto peer_address = peer_endpoint.address().is_v6() ? IPAddress(peer_endpoint.address().to_v6().to_bytes()) + : IPAddress(peer_endpoint.address().to_v4().to_ulong()); + conn->accept(NetworkAddress(peer_address, peer_endpoint.port())); + + return conn; + } catch (...) { + conn->close(); + throw; + } + } +}; + +#ifndef TLS_DISABLED +typedef boost::asio::ssl::stream ssl_socket; + class SSLConnection : public IConnection, ReferenceCounted { public: virtual void addref() { ReferenceCounted::addref(); } @@ -732,48 +781,6 @@ private: } }; -class Listener : public IListener, ReferenceCounted { - NetworkAddress listenAddress; - tcp::acceptor acceptor; - -public: - Listener( boost::asio::io_service& io_service, NetworkAddress listenAddress ) - : listenAddress(listenAddress), acceptor( io_service, tcpEndpoint( listenAddress ) ) - { - platform::setCloseOnExec(acceptor.native_handle()); - } - - virtual void addref() { ReferenceCounted::addref(); } - virtual void delref() { ReferenceCounted::delref(); } - - // Returns one incoming connection when it is available - virtual Future> accept() { - return doAccept( this ); - } - - virtual NetworkAddress getListenAddress() { return listenAddress; } - -private: - ACTOR static Future> doAccept( Listener* self ) { - state Reference conn( new Connection( self->acceptor.get_io_service() ) ); - state tcp::acceptor::endpoint_type peer_endpoint; - try { - BindPromise p("N2_AcceptError", UID()); - auto f = p.getFuture(); - self->acceptor.async_accept( conn->getSocket(), peer_endpoint, std::move(p) ); - wait( f ); - auto peer_address = peer_endpoint.address().is_v6() ? IPAddress(peer_endpoint.address().to_v6().to_bytes()) - : IPAddress(peer_endpoint.address().to_v4().to_ulong()); - conn->accept(NetworkAddress(peer_address, peer_endpoint.port())); - - return conn; - } catch (...) { - conn->close(); - throw; - } - } -}; - class SSLListener : public IListener, ReferenceCounted { NetworkAddress listenAddress; tcp::acceptor acceptor; @@ -816,6 +823,7 @@ private: } } }; +#endif struct PromiseTask : public Task, public FastAllocated { Promise promise; @@ -832,9 +840,11 @@ struct PromiseTask : public Task, public FastAllocated { // TODO: Move to a headerfile and delete all the copies of this. #define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) +#ifndef TLS_DISABLED bool insecurely_always_accept(bool _1, boost::asio::ssl::verify_context& _2) { return true; } +#endif Net2::Net2(bool useThreadPool, bool useMetrics, Reference policy, const TLSParams& tlsParams) : useThreadPool(useThreadPool), @@ -846,12 +856,15 @@ Net2::Net2(bool useThreadPool, bool useMetrics, Reference policy, con tsc_begin(0), tsc_end(0), taskBegin(0), currentTaskID(TaskPriority::DefaultYield), lastMinTaskID(TaskPriority::Zero), numYields(0), - tlsPassword(tlsParams.tlsPassword), - sslContext(boost::asio::ssl::context(boost::asio::ssl::context::tlsv12)) + tlsPassword(tlsParams.tlsPassword) +#ifndef TLS_DISABLED + ,sslContext(boost::asio::ssl::context(boost::asio::ssl::context::tlsv12)) +#endif { TraceEvent("Net2Starting"); +#ifndef TLS_DISABLED sslContext.set_options(boost::asio::ssl::context::default_workarounds); sslContext.set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert); if (policy) { @@ -883,6 +896,7 @@ Net2::Net2(bool useThreadPool, bool useMetrics, Reference policy, con if (tlsParams.tlsKeyBytes.size()) { sslContext.use_private_key(boost::asio::buffer(tlsParams.tlsKeyBytes.data(), tlsParams.tlsKeyBytes.size()), boost::asio::ssl::context::pem); } +#endif // Set the global members if(useMetrics) { @@ -1260,9 +1274,11 @@ THREAD_HANDLE Net2::startThread( THREAD_FUNC_RETURN (*func) (void*), void *arg ) } Future< Reference > Net2::connect( NetworkAddress toAddr, std::string host ) { +#ifndef TLS_DISABLED if ( toAddr.isTLS() ) { return SSLConnection::connect(&this->reactor.ios, &this->sslContext, toAddr); } +#endif return Connection::connect(&this->reactor.ios, toAddr); } @@ -1337,9 +1353,11 @@ bool Net2::isAddressOnThisHost( NetworkAddress const& addr ) { Reference Net2::listen( NetworkAddress localAddr ) { try { +#ifndef TLS_DISABLED if ( localAddr.isTLS() ) { return Reference(new SSLListener( reactor.ios, &this->sslContext, localAddr )); } +#endif return Reference( new Listener( reactor.ios, localAddr ) ); } catch (boost::system::system_error const& e) { Error x; diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp index e636c4af8d..5ac2b5d78d 100644 --- a/flow/TLSPolicy.cpp +++ b/flow/TLSPolicy.cpp @@ -18,6 +18,11 @@ * limitations under the License. */ +#include "flow/TLSPolicy.h" + +TLSPolicy::~TLSPolicy() {} + +#ifndef TLS_DISABLED #include #include #include @@ -35,7 +40,6 @@ #include "flow/FastRef.h" #include "flow/Trace.h" -#include "flow/TLSPolicy.h" std::string TLSPolicy::ErrorString(boost::system::error_code e) { char* str = ERR_error_string(e.value(), NULL); @@ -43,7 +47,7 @@ std::string TLSPolicy::ErrorString(boost::system::error_code e) { } // To force typeinfo to only be emitted once. -TLSPolicy::~TLSPolicy() {} + std::string TLSPolicy::toString() const { std::stringstream ss; @@ -526,3 +530,4 @@ bool TLSPolicy::verify_peer(bool preverified, X509_STORE_CTX* store_ctx) { } return rc; } +#endif diff --git a/flow/TLSPolicy.h b/flow/TLSPolicy.h index a50fa879e3..1af5abfb73 100644 --- a/flow/TLSPolicy.h +++ b/flow/TLSPolicy.h @@ -25,17 +25,12 @@ #include #include #include -#include #include #include "flow/FastRef.h" -struct TLSParams { - enum { OPT_TLS = 100000, OPT_TLS_PLUGIN, OPT_TLS_CERTIFICATES, OPT_TLS_KEY, OPT_TLS_VERIFY_PEERS, OPT_TLS_CA_FILE, OPT_TLS_PASSWORD }; - - std::string tlsCertPath, tlsKeyPath, tlsCAPath, tlsPassword; - std::string tlsCertBytes, tlsKeyBytes, tlsCABytes; -}; +#ifndef TLS_DISABLED +#include typedef int NID; enum class MatchType { @@ -69,6 +64,14 @@ struct Criteria { return criteria == c.criteria && match_type == c.match_type && location == c.location; } }; +#endif + +struct TLSParams { + enum { OPT_TLS = 100000, OPT_TLS_PLUGIN, OPT_TLS_CERTIFICATES, OPT_TLS_KEY, OPT_TLS_VERIFY_PEERS, OPT_TLS_CA_FILE, OPT_TLS_PASSWORD }; + + std::string tlsCertPath, tlsKeyPath, tlsCAPath, tlsPassword; + std::string tlsCertBytes, tlsKeyBytes, tlsCABytes; +}; class TLSPolicy : ReferenceCounted { public: @@ -83,6 +86,7 @@ public: virtual void addref() { ReferenceCounted::addref(); } virtual void delref() { ReferenceCounted::delref(); } +#ifndef TLS_DISABLED static std::string ErrorString(boost::system::error_code e); bool set_verify_peers(std::vector verify_peers); @@ -104,6 +108,7 @@ public: }; std::vector rules; +#endif bool is_client; }; diff --git a/flow/network.h b/flow/network.h index a9353e6d0f..555203f88d 100644 --- a/flow/network.h +++ b/flow/network.h @@ -27,7 +27,9 @@ #include #include #include "boost/asio.hpp" +#ifndef TLS_DISABLED #include "boost/asio/ssl.hpp" +#endif #include "flow/serialize.h" #include "flow/IRandom.h" #include "flow/TLSPolicy.h" From 3c4d5516479412f82d46ef63557d1b9b63b77b38 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Wed, 19 Feb 2020 18:50:21 -0800 Subject: [PATCH 27/44] improve prioritization of connection monitor and listen given that listen is no longer expensive (because handshake is done separately) --- fdbrpc/FlowTransport.actor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 36238f93a9..1edae87068 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -302,7 +302,7 @@ ACTOR Future connectionMonitor( Reference peer ) { state double lastRefreshed = now(); state int64_t lastBytesReceived = peer->bytesReceived; loop { - wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_LOOP_TIME)); + wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_LOOP_TIME, TaskPriority::ReadSocket)); if (lastBytesReceived < peer->bytesReceived) { lastRefreshed = now(); lastBytesReceived = peer->bytesReceived; @@ -317,7 +317,7 @@ ACTOR Future connectionMonitor( Reference peer ) { //We cannot let an error be thrown from connectionMonitor while still on the stack from scanPackets in connectionReader //because then it would not call the destructor of connectionReader when connectionReader is cancelled. - wait(delay(0)); + wait(delay(0, TaskPriority::ReadSocket)); if (peer->reliable.empty() && peer->unsent.empty() && peer->outstandingReplies==0) { if (peer->peerReferences == 0 && @@ -332,7 +332,7 @@ ACTOR Future connectionMonitor( Reference peer ) { } } - wait (delayJittered(FLOW_KNOBS->CONNECTION_MONITOR_LOOP_TIME)); + wait (delayJittered(FLOW_KNOBS->CONNECTION_MONITOR_LOOP_TIME, TaskPriority::ReadSocket)); // TODO: Stop monitoring and close the connection with no onDisconnect requests outstanding state ReplyPromise reply; @@ -997,7 +997,7 @@ ACTOR static Future listen( TransportData* self, NetworkAddress listenAddr .detail("ListenAddress", listenAddr.toString()); incoming.add( connectionIncoming(self, conn) ); } - wait(delay(0) || delay(FLOW_KNOBS->CONNECTION_ACCEPT_DELAY, TaskPriority::WriteSocket)); + wait(delay(0, decrementPriority(TaskPriority::ReadSocket))); } } catch (Error& e) { TraceEvent(SevError, "ListenError").error(e); From 69b5a1fbe30e76e99223af7142c404382f5c79d8 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Feb 2020 10:11:43 -0800 Subject: [PATCH 28/44] more priority improvements --- flow/Knobs.cpp | 1 - flow/Knobs.h | 1 - flow/Net2.actor.cpp | 4 ++-- flow/network.h | 1 + 4 files changed, 3 insertions(+), 4 deletions(-) diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index 7dd802e0d6..c2a478e7f9 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -67,7 +67,6 @@ FlowKnobs::FlowKnobs(bool randomize, bool isSimulated) { init( MAX_RECONNECTION_TIME, 0.5 ); init( RECONNECTION_TIME_GROWTH_RATE, 1.2 ); init( RECONNECTION_RESET_TIME, 5.0 ); - init( CONNECTION_ACCEPT_DELAY, 0.5 ); init( USE_OBJECT_SERIALIZER, 1 ); init( TOO_MANY_CONNECTIONS_CLOSED_RESET_DELAY, 5.0 ); init( TOO_MANY_CONNECTIONS_CLOSED_TIMEOUT, 20.0 ); diff --git a/flow/Knobs.h b/flow/Knobs.h index fc72b3cf44..6ecf1c8759 100644 --- a/flow/Knobs.h +++ b/flow/Knobs.h @@ -87,7 +87,6 @@ public: double MAX_RECONNECTION_TIME; double RECONNECTION_TIME_GROWTH_RATE; double RECONNECTION_RESET_TIME; - double CONNECTION_ACCEPT_DELAY; int USE_OBJECT_SERIALIZER; int TLS_CERT_REFRESH_DELAY_SECONDS; diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 4fdc769b76..a476ed2352 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -569,7 +569,7 @@ public: auto onHandshook = p.getFuture(); self->getSSLSocket().async_handshake( boost::asio::ssl::stream_base::server, std::move(p) ); wait( onHandshook ); - wait(delay(0) || delay(FLOW_KNOBS->CONNECTION_ACCEPT_DELAY, TaskPriority::WriteSocket)); + wait(delay(0, TaskPriority::Handshake)); connected.send(Void()); } catch (...) { auto iter(g_network->networkInfo.serverTLSConnectionThrottler.find(peerIP)); @@ -609,7 +609,7 @@ public: Future onHandshook = p.getFuture(); self->ssl_sock.async_handshake( boost::asio::ssl::stream_base::client, std::move(p) ); wait( onHandshook ); - wait(delay(0) || delay(FLOW_KNOBS->CONNECTION_ACCEPT_DELAY, TaskPriority::WriteSocket)); + wait(delay(0, TaskPriority::Handshake)); connected.send(Void()); } catch (...) { std::pair peerIP = std::make_pair(self->peer_address.ip, self->peer_address.port); diff --git a/flow/network.h b/flow/network.h index 555203f88d..cd61040b51 100644 --- a/flow/network.h +++ b/flow/network.h @@ -44,6 +44,7 @@ enum class TaskPriority { DiskIOComplete = 9150, LoadBalancedEndpoint = 9000, ReadSocket = 9000, + Handshake = 8900, CoordinationReply = 8810, Coordination = 8800, FailureMonitor = 8700, From 08c318d28a4d730f43cd0151dceac7f372c885e4 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Feb 2020 10:43:34 -0800 Subject: [PATCH 29/44] re-added the connect lock in the fdbcli so that the timeout is not spent before a connection has been initiated (because of the handshake lock) --- fdbcli/fdbcli.actor.cpp | 12 ++++++++---- fdbclient/Knobs.cpp | 4 ++++ fdbclient/Knobs.h | 4 ++++ fdbclient/NativeAPI.actor.cpp | 2 -- fdbserver/fdbserver.actor.cpp | 2 -- flow/Knobs.cpp | 1 + flow/Knobs.h | 1 + flow/Net2.actor.cpp | 4 +--- 8 files changed, 19 insertions(+), 11 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index baf38a2afd..c8c3ba9e1a 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -2552,7 +2552,9 @@ Future stopNetworkAfter( Future what ) { } } -ACTOR Future addInterface( std::map>* address_interface, KeyValue kv) { +ACTOR Future addInterface( std::map>* address_interface, Reference connectLock, KeyValue kv) { + wait(connectLock->take()); + state FlowLock::Releaser releaser(*connectLock); state ClientWorkerInterface workerInterf = BinaryReader::fromStringRef(kv.value, IncludeVersion()); state ClientLeaderRegInterface leaderInterf(workerInterf.address()); choose { @@ -2566,7 +2568,7 @@ ACTOR Future addInterface( std::mapCLI_CONNECT_TIMEOUT)) ) {} } return Void(); } @@ -2974,9 +2976,10 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { getTransaction(db, tr, options, intrans); if (tokens.size() == 1) { Standalone kvs = wait( makeInterruptable( tr->getRange(KeyRangeRef(LiteralStringRef("\xff\xff/worker_interfaces"), LiteralStringRef("\xff\xff\xff")), 1) ) ); + Reference connectLock(new FlowLock(CLIENT_KNOBS->CLI_CONNECT_PARALLELISM)); std::vector> addInterfs; for( auto it : kvs ) { - addInterfs.push_back(addInterface(&address_interface, it)); + addInterfs.push_back(addInterface(&address_interface, connectLock, it)); } wait( waitForAll(addInterfs) ); } @@ -3287,9 +3290,10 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { getTransaction(db, tr, options, intrans); if (tokens.size() == 1) { Standalone kvs = wait( makeInterruptable( tr->getRange(KeyRangeRef(LiteralStringRef("\xff\xff/worker_interfaces"), LiteralStringRef("\xff\xff\xff")), 1) ) ); + Reference connectLock(new FlowLock(CLIENT_KNOBS->CLI_CONNECT_PARALLELISM)); std::vector> addInterfs; for( auto it : kvs ) { - addInterfs.push_back(addInterface(&address_interface, it)); + addInterfs.push_back(addInterface(&address_interface, connectLock, it)); } wait( waitForAll(addInterfs) ); } diff --git a/fdbclient/Knobs.cpp b/fdbclient/Knobs.cpp index 7b3cba805f..c8b1472259 100644 --- a/fdbclient/Knobs.cpp +++ b/fdbclient/Knobs.cpp @@ -200,4 +200,8 @@ ClientKnobs::ClientKnobs(bool randomize) { init( CONSISTENCY_CHECK_RATE_LIMIT_MAX, 50e6 ); // Limit in per sec init( CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME, 7 * 24 * 60 * 60 ); // 7 days + + //fdbcli + init( CLI_CONNECT_PARALLELISM, 400 ); + init( CLI_CONNECT_TIMEOUT, 10.0 ); } diff --git a/fdbclient/Knobs.h b/fdbclient/Knobs.h index 359e40acde..56772229ad 100644 --- a/fdbclient/Knobs.h +++ b/fdbclient/Knobs.h @@ -190,6 +190,10 @@ public: int CONSISTENCY_CHECK_RATE_LIMIT_MAX; int CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME; + + // fdbcli + int CLI_CONNECT_PARALLELISM; + double CLI_CONNECT_TIMEOUT; ClientKnobs(bool randomize = false); }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 9b1aaec603..fbf09ca3d3 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -66,8 +66,6 @@ using std::max; using std::min; using std::pair; -#define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) - NetworkOptions networkOptions; TLSParams tlsParams; static Reference tlsPolicy; diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index 233eaba14b..5448bfa161 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -81,8 +81,6 @@ #include "flow/SimpleOpt.h" #include "flow/actorcompiler.h" // This must be the last #include. -#define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) - enum { OPT_CONNFILE, OPT_SEEDCONNFILE, OPT_SEEDCONNSTRING, OPT_ROLE, OPT_LISTEN, OPT_PUBLICADDR, OPT_DATAFOLDER, OPT_LOGFOLDER, OPT_PARENTPID, OPT_NEWCONSOLE, OPT_NOBOX, OPT_TESTFILE, OPT_RESTARTING, OPT_RESTORING, OPT_RANDOMSEED, OPT_KEY, OPT_MEMLIMIT, OPT_STORAGEMEMLIMIT, OPT_CACHEMEMLIMIT, OPT_MACHINEID, diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index c2a478e7f9..82dfdb2c98 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -115,6 +115,7 @@ FlowKnobs::FlowKnobs(bool randomize, bool isSimulated) { init( SLOW_LOOP_CUTOFF, 15.0 / 1000.0 ); init( SLOW_LOOP_SAMPLING_RATE, 0.1 ); init( TSC_YIELD_TIME, 1000000 ); + init( CERT_FILE_MAX_SIZE, 5 * 1024 * 1024 ); //Network init( PACKET_LIMIT, 100LL<<20 ); diff --git a/flow/Knobs.h b/flow/Knobs.h index 6ecf1c8759..8d6223ab08 100644 --- a/flow/Knobs.h +++ b/flow/Knobs.h @@ -136,6 +136,7 @@ public: double SLOW_LOOP_SAMPLING_RATE; int64_t TSC_YIELD_TIME; int64_t REACTOR_FLAGS; + int CERT_FILE_MAX_SIZE; //Network int64_t PACKET_LIMIT; diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index a476ed2352..cbaa4b322a 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -837,8 +837,6 @@ struct PromiseTask : public Task, public FastAllocated { }; // 5MB for loading files into memory -// TODO: Move to a headerfile and delete all the copies of this. -#define CERT_FILE_MAX_SIZE (5 * 1024 * 1024) #ifndef TLS_DISABLED bool insecurely_always_accept(bool _1, boost::asio::ssl::verify_context& _2) { @@ -884,7 +882,7 @@ Net2::Net2(bool useThreadPool, bool useMetrics, Reference policy, con sslContext.use_certificate(boost::asio::buffer(tlsParams.tlsCertBytes.data(), tlsParams.tlsCertBytes.size()), boost::asio::ssl::context::pem); } if (tlsParams.tlsCAPath.size()) { - std::string cert = readFileBytes(tlsParams.tlsCAPath, CERT_FILE_MAX_SIZE); + std::string cert = readFileBytes(tlsParams.tlsCAPath, FLOW_KNOBS->CERT_FILE_MAX_SIZE); sslContext.add_certificate_authority(boost::asio::buffer(cert.data(), cert.size())); } if (tlsParams.tlsCABytes.size()) { From f7a37077cc72cf0acf3945883c1513152202a34f Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Feb 2020 15:26:56 -0800 Subject: [PATCH 30/44] handshake takes time in simulation --- fdbrpc/sim2.actor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 918f0510bb..d662271393 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -200,8 +200,8 @@ struct Sim2Conn : IConnection, ReferenceCounted { virtual void delref() { ReferenceCounted::delref(); } virtual void close() { closedByCaller = true; closeInternal(); } - virtual Future acceptHandshake() { return Void(); } - virtual Future connectHandshake() { return Void(); } + virtual Future acceptHandshake() { return delay(0.01*deterministicRandom()->random01()); } + virtual Future connectHandshake() { return delay(0.01*deterministicRandom()->random01()); } virtual Future onWritable() { return whenWritable(this); } virtual Future onReadable() { return whenReadable(this); } From 927cff33179c1b72801d977dc385ff6ed68360e7 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 20 Feb 2020 16:53:01 -0800 Subject: [PATCH 31/44] Report errors on TLS misconfigurations ... or at least try to. --- bindings/c/fdb_c.cpp | 7 ++++++- fdbbackup/backup.actor.cpp | 7 +++++++ fdbcli/fdbcli.actor.cpp | 3 +++ fdbserver/fdbserver.actor.cpp | 5 +++++ flow/Net2.actor.cpp | 2 +- 5 files changed, 22 insertions(+), 2 deletions(-) diff --git a/bindings/c/fdb_c.cpp b/bindings/c/fdb_c.cpp index 356c3225d5..1c787f060a 100644 --- a/bindings/c/fdb_c.cpp +++ b/bindings/c/fdb_c.cpp @@ -108,7 +108,12 @@ fdb_error_t fdb_network_set_option( FDBNetworkOption option, } fdb_error_t fdb_setup_network_impl() { - CATCH_AND_RETURN( API->setupNetwork(); ); + CATCH_AND_RETURN( + try { + API->setupNetwork(); + } catch (boost::system::system_error& e) { + return error_code_tls_error; + } ); } fdb_error_t fdb_setup_network_v13( const char* localAddress ) { diff --git a/fdbbackup/backup.actor.cpp b/fdbbackup/backup.actor.cpp index d120fa06da..2ea44f1a99 100644 --- a/fdbbackup/backup.actor.cpp +++ b/fdbbackup/backup.actor.cpp @@ -3657,6 +3657,13 @@ int main(int argc, char* argv[]) { } catch (Error& e) { TraceEvent(SevError, "MainError").error(e); status = FDB_EXIT_MAIN_ERROR; + } catch (boost::system::system_error& e) { + if (g_network) { + TraceEvent(SevError, "MainError").error(unknown_error()).detail("RootException", e.what()); + } else { + fprintf(stderr, "ERROR: %s (%d)\n", e.what(), e.code().value()); + } + status = FDB_EXIT_MAIN_EXCEPTION; } catch (std::exception& e) { TraceEvent(SevError, "MainError").error(unknown_error()).detail("RootException", e.what()); status = FDB_EXIT_MAIN_EXCEPTION; diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index c8c3ba9e1a..4f7cf83972 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3753,5 +3753,8 @@ int main(int argc, char **argv) { } catch (Error& e) { printf("ERROR: %s (%d)\n", e.what(), e.code()); return 1; + } catch (boost::system::system_error& e) { + printf("ERROR: %s (%d)\n", e.what(), e.code().value()); + return 1; } } diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index 5448bfa161..9df3ea4cf5 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -1963,6 +1963,11 @@ int main(int argc, char* argv[]) { TraceEvent(SevError, "MainError").error(e); //printf("\n%d tests passed; %d tests failed\n", passCount, failCount); flushAndExit(FDB_EXIT_MAIN_ERROR); + } catch (boost::system::system_error& e) { + fprintf(stderr, "boost::system::system_error: %s (%d)", e.what(), e.code().value()); + TraceEvent(SevError, "MainError").error(unknown_error()).detail("RootException", e.what()); + //printf("\n%d tests passed; %d tests failed\n", passCount, failCount); + flushAndExit(FDB_EXIT_MAIN_EXCEPTION); } catch (std::exception& e) { fprintf(stderr, "std::exception: %s\n", e.what()); TraceEvent(SevError, "MainError").error(unknown_error()).detail("RootException", e.what()); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index cbaa4b322a..f922eea985 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -1456,7 +1456,7 @@ INetwork* newNet2(bool useThreadPool, bool useMetrics, Reference poli } catch(boost::system::system_error e) { TraceEvent("Net2InitError").detail("Message", e.what()); - throw unknown_error(); + throw; } catch(std::exception const& e) { TraceEvent("Net2InitError").detail("Message", e.what()); From 41afeb245ee3dd4f4df4ae2d12d6fe9202e0df4b Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Thu, 20 Feb 2020 17:03:16 -0800 Subject: [PATCH 32/44] Temporarily disable TLS on OSX --- Makefile | 3 +++ cmake/FDBComponents.cmake | 6 +++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 30d70a81b5..a078a5f838 100644 --- a/Makefile +++ b/Makefile @@ -64,6 +64,9 @@ else ifeq ($(PLATFORM),Darwin) .LIBPATTERNS := lib%.dylib lib%.a + # Temporarily disable TLS on OSX as libressl -> openssl transition happens + TLS_DISABLED := 1 + BOOST_BASEDIR ?= ${HOME} TLS_LIBDIR ?= /usr/local/lib DLEXT := dylib diff --git a/cmake/FDBComponents.cmake b/cmake/FDBComponents.cmake index d6fe00a598..37b4d9af39 100644 --- a/cmake/FDBComponents.cmake +++ b/cmake/FDBComponents.cmake @@ -25,7 +25,7 @@ else() add_library(OpenSSL::SSL ALIAS LibreSSL) endif() endif() - if(OPENSSL_FOUND OR LIBRESSL_FOUND) + if(OPENSSL_FOUND OR LIBRESSL_FOUND) set(WITH_TLS ON) add_compile_options(-DHAVE_OPENSSL) else() @@ -33,6 +33,10 @@ else() message(STATUS "You can set OPENSSL_ROOT_DIR or LibreSSL_ROOT to the LibreSSL install directory to help cmake find it") set(WITH_TLS OFF) endif() + if(APPLE) + message(STATUS "TLS is temporarilty disabled on macOS while libressl -> openssl transition happens") + set(WITH_TLS OFF) + endif() endif() ################################################################################ From 3bef06dd471327ffd74ecc677410b49dce14a899 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Feb 2020 17:20:48 -0800 Subject: [PATCH 33/44] TLS_DISABLED also implies we do not have openssl --- fdbclient/md5/md5.c | 2 +- fdbclient/md5/md5.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fdbclient/md5/md5.c b/fdbclient/md5/md5.c index 52d96accd3..b331c91a78 100644 --- a/fdbclient/md5/md5.c +++ b/fdbclient/md5/md5.c @@ -35,7 +35,7 @@ * compile-time configuration. */ -#ifndef HAVE_OPENSSL +#if defined(HAVE_OPENSSL) && !defined(TLS_DISABLED) #include diff --git a/fdbclient/md5/md5.h b/fdbclient/md5/md5.h index e73fb29c35..5731872376 100644 --- a/fdbclient/md5/md5.h +++ b/fdbclient/md5/md5.h @@ -23,7 +23,7 @@ * See md5.c for more information. */ -#ifdef HAVE_OPENSSL +#if defined(HAVE_OPENSSL) && !defined(TLS_DISABLED) #include #elif !defined(_MD5_H) #define _MD5_H From efbc8141a0555cdc0d028885a965334ce69f734f Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Feb 2020 17:29:06 -0800 Subject: [PATCH 34/44] fix: messed up define --- fdbclient/md5/md5.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbclient/md5/md5.c b/fdbclient/md5/md5.c index b331c91a78..1032ccfdaf 100644 --- a/fdbclient/md5/md5.c +++ b/fdbclient/md5/md5.c @@ -35,7 +35,7 @@ * compile-time configuration. */ -#if defined(HAVE_OPENSSL) && !defined(TLS_DISABLED) +#if !defined(HAVE_OPENSSL) || defined(TLS_DISABLED) #include From 3a3f6afc4c75d48867286ff53c564b4bf2f224b3 Mon Sep 17 00:00:00 2001 From: Evan Tschannen <36455792+etschannen@users.noreply.github.com> Date: Thu, 20 Feb 2020 18:04:44 -0800 Subject: [PATCH 35/44] Update fdbcli/fdbcli.actor.cpp --- fdbcli/fdbcli.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 8d7a702c9e..223a624a75 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -2557,7 +2557,7 @@ Future stopNetworkAfter( Future what ) { } ACTOR Future addInterface( std::map>* address_interface, Reference connectLock, KeyValue kv) { - wait(connectLock->take()); + wait(connectLock->take()); state FlowLock::Releaser releaser(*connectLock); state ClientWorkerInterface workerInterf = BinaryReader::fromStringRef(kv.value, IncludeVersion()); state ClientLeaderRegInterface leaderInterf(workerInterf.address()); From dc3826e2fd4351a100aafd6453c785a32ab41688 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Feb 2020 18:17:39 -0800 Subject: [PATCH 36/44] fix: tls throttling would re-insert the failure into the map --- fdbrpc/FlowTransport.actor.cpp | 2 +- flow/Net2.actor.cpp | 19 +++++++++---------- flow/network.h | 1 + 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index 7ac1ace197..b00ca240c7 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -997,7 +997,7 @@ ACTOR static Future listen( TransportData* self, NetworkAddress listenAddr .detail("ListenAddress", listenAddr.toString()); incoming.add( connectionIncoming(self, conn) ); } - wait(delay(0, decrementPriority(TaskPriority::ReadSocket))); + wait(delay(0, TaskPriority::AcceptSocket)); } } catch (Error& e) { TraceEvent(SevError, "ListenError").error(e); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 214dc35ac2..22a58b181e 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -556,7 +556,9 @@ public: if(iter->second.first >= FLOW_KNOBS->TLS_SERVER_CONNECTION_THROTTLE_ATTEMPTS) { TraceEvent("TLSIncomingConnectionThrottlingWarning").suppressFor(1.0).detail("PeerIP", peerIP.first.toString()); wait(delay(FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT)); - throw connection_failed(); + self->closeSocket(); + connected.sendError(connection_failed()); + return; } } else { g_network->networkInfo.serverTLSConnectionThrottler.erase(peerIP); @@ -761,15 +763,12 @@ private: } void closeSocket() { - try { - socket.cancel(); - } catch(...) {} - try { - socket.close(); - } catch(...) {} - try { - ssl_sock.shutdown(); - } catch(...) {} + boost::system::error_code cancelError; + socket.cancel(cancelError); + boost::system::error_code closeError; + socket.close(closeError); + boost::system::error_code shutdownError; + ssl_sock.shutdown(shutdownError); } void onReadError( const boost::system::error_code& error ) { diff --git a/flow/network.h b/flow/network.h index 02898797dd..d3dd9aa026 100644 --- a/flow/network.h +++ b/flow/network.h @@ -44,6 +44,7 @@ enum class TaskPriority { DiskIOComplete = 9150, LoadBalancedEndpoint = 9000, ReadSocket = 9000, + AcceptSocket = 8950, Handshake = 8900, CoordinationReply = 8810, Coordination = 8800, From 0e4df0004d65e76ed11e16fed4aefe9e0cee4778 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Thu, 20 Feb 2020 20:05:45 -0800 Subject: [PATCH 37/44] re-enable tls for macos --- Makefile | 3 --- 1 file changed, 3 deletions(-) diff --git a/Makefile b/Makefile index a078a5f838..30d70a81b5 100644 --- a/Makefile +++ b/Makefile @@ -64,9 +64,6 @@ else ifeq ($(PLATFORM),Darwin) .LIBPATTERNS := lib%.dylib lib%.a - # Temporarily disable TLS on OSX as libressl -> openssl transition happens - TLS_DISABLED := 1 - BOOST_BASEDIR ?= ${HOME} TLS_LIBDIR ?= /usr/local/lib DLEXT := dylib From 8f52fc77c2154b9b52498b9fcd82c33ac9ec0f55 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 21 Feb 2020 08:18:59 -0800 Subject: [PATCH 38/44] updating docker version --- build/docker-compose.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build/docker-compose.yaml b/build/docker-compose.yaml index 1853cc80fd..4703cd7eda 100644 --- a/build/docker-compose.yaml +++ b/build/docker-compose.yaml @@ -2,7 +2,7 @@ version: "3" services: common: &common - image: foundationdb/foundationdb-build:0.1.9 + image: foundationdb/foundationdb-build:0.1.11 build-setup: &build-setup <<: *common From 735c3dadac99ea15d27e26b4b76cda8a4705d037 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 21 Feb 2020 08:31:52 -0800 Subject: [PATCH 39/44] updated makefile --- Makefile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Makefile b/Makefile index 30d70a81b5..1a04e14bd4 100644 --- a/Makefile +++ b/Makefile @@ -49,7 +49,7 @@ ifeq ($(PLATFORM),Linux) CXXFLAGS += -std=c++17 BOOST_BASEDIR ?= /opt - TLS_LIBDIR ?= /usr/local/lib + TLS_LIBDIR ?= /usr/local/lib64 DLEXT := so java_DLEXT := so TARGET_LIBC_VERSION ?= 2.11 @@ -65,7 +65,7 @@ else ifeq ($(PLATFORM),Darwin) .LIBPATTERNS := lib%.dylib lib%.a BOOST_BASEDIR ?= ${HOME} - TLS_LIBDIR ?= /usr/local/lib + TLS_LIBDIR ?= /usr/local/lib64 DLEXT := dylib java_DLEXT := jnilib else From 87751df40a6ad684e3a915ad776d72b937c925a3 Mon Sep 17 00:00:00 2001 From: Alvin Moore Date: Fri, 21 Feb 2020 08:45:39 -0800 Subject: [PATCH 40/44] Fixed problem with linking pthread --- fdbserver/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index d4c5e5f226..bbe3c686ce 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -201,6 +201,9 @@ target_include_directories(fdbserver PRIVATE ${CMAKE_CURRENT_BINARY_DIR}/workloads ${CMAKE_CURRENT_SOURCE_DIR}/workloads) target_link_libraries(fdbserver PRIVATE fdbclient fdb_sqlite) +if (HAVE_PTHREAD) + target_link_libraries (fdbserver SYSTEM PUBLIC ${CMAKE_THREAD_LIBS_INIT}) +endif (HAVE_PTHREAD) if (GPERFTOOLS_FOUND) add_compile_definitions(USE_GPERFTOOLS) target_link_libraries(fdbserver PRIVATE gperftools) From 9042cab7bc4215b3a8ddbb2678e0e5da09c9d9f4 Mon Sep 17 00:00:00 2001 From: Alvin Moore Date: Fri, 21 Feb 2020 08:56:52 -0800 Subject: [PATCH 41/44] Changed ordering of link libraries --- fdbserver/CMakeLists.txt | 3 --- flow/CMakeLists.txt | 8 ++++---- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index bbe3c686ce..d4c5e5f226 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -201,9 +201,6 @@ target_include_directories(fdbserver PRIVATE ${CMAKE_CURRENT_BINARY_DIR}/workloads ${CMAKE_CURRENT_SOURCE_DIR}/workloads) target_link_libraries(fdbserver PRIVATE fdbclient fdb_sqlite) -if (HAVE_PTHREAD) - target_link_libraries (fdbserver SYSTEM PUBLIC ${CMAKE_THREAD_LIBS_INIT}) -endif (HAVE_PTHREAD) if (GPERFTOOLS_FOUND) add_compile_definitions(USE_GPERFTOOLS) target_link_libraries(fdbserver PRIVATE gperftools) diff --git a/flow/CMakeLists.txt b/flow/CMakeLists.txt index dbccb2f621..4af884bb8d 100644 --- a/flow/CMakeLists.txt +++ b/flow/CMakeLists.txt @@ -92,15 +92,15 @@ elseif(WIN32) target_link_libraries(flow PUBLIC psapi.lib) endif() target_link_libraries(flow PRIVATE ${FLOW_LIBS}) -target_link_libraries(flow PUBLIC boost_target Threads::Threads ${CMAKE_DL_LIBS}) -if(USE_VALGRIND) - target_link_libraries(flow PUBLIC Valgrind) -endif() if(NOT WITH_TLS) target_compile_definitions(flow PUBLIC TLS_DISABLED) else() target_link_libraries(flow PUBLIC OpenSSL::SSL) endif() +target_link_libraries(flow PUBLIC boost_target Threads::Threads ${CMAKE_DL_LIBS}) +if(USE_VALGRIND) + target_link_libraries(flow PUBLIC Valgrind) +endif() if(APPLE) find_library(IO_KIT IOKit) From d02d84a57713d994d00659c80e8573a9934043ee Mon Sep 17 00:00:00 2001 From: Alvin Moore Date: Fri, 21 Feb 2020 09:36:24 -0800 Subject: [PATCH 42/44] Added required include for std:set which is for some reason only missing within Windows build --- flow/TLSPolicy.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp index 5ac2b5d78d..949e93a2d1 100644 --- a/flow/TLSPolicy.cpp +++ b/flow/TLSPolicy.cpp @@ -27,6 +27,7 @@ TLSPolicy::~TLSPolicy() {} #include #include #include +#include #include #include #include From 90b4050eca17aefb75dae1c96a17e7c317cc458e Mon Sep 17 00:00:00 2001 From: Alvin Moore Date: Fri, 21 Feb 2020 09:59:11 -0800 Subject: [PATCH 43/44] Added required include for stringstream --- flow/TLSPolicy.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/flow/TLSPolicy.cpp b/flow/TLSPolicy.cpp index 949e93a2d1..c3a71abe1e 100644 --- a/flow/TLSPolicy.cpp +++ b/flow/TLSPolicy.cpp @@ -37,6 +37,7 @@ TLSPolicy::~TLSPolicy() {} #include #include #include +#include #include #include "flow/FastRef.h" From 3354e6b2780596873b2278a6f9a36c45b7b9a440 Mon Sep 17 00:00:00 2001 From: Alvin Moore Date: Fri, 21 Feb 2020 11:23:51 -0800 Subject: [PATCH 44/44] Enabled TLS for CMake Mac builds Disabled TLS for CMake Windows builds --- cmake/FDBComponents.cmake | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/FDBComponents.cmake b/cmake/FDBComponents.cmake index aebec99cdc..65623f31bc 100644 --- a/cmake/FDBComponents.cmake +++ b/cmake/FDBComponents.cmake @@ -33,10 +33,10 @@ else() message(STATUS "You can set OPENSSL_ROOT_DIR or LibreSSL_ROOT to the LibreSSL install directory to help cmake find it") set(WITH_TLS OFF) endif() - if(APPLE) - message(STATUS "TLS is temporarilty disabled on macOS while libressl -> openssl transition happens") - set(WITH_TLS OFF) - endif() + if(WIN32) + message(STATUS "TLS is temporarilty disabled on macOS while libressl -> openssl transition happens") + set(WITH_TLS OFF) + endif() endif() ################################################################################