Switch to rustls and webpki-roots (#30025)

This change replaces OpenSSL with rustls and also the manually curated
CA certs file with webpki-roots (effectively the same thing, but as a
crate).

Generally speaking the design of the network stack is the same. Changes:

- Code around certificate overrides needed to be refactored to work with
  rustls so the various thread-safe list of certificates is refactored
  into `CertificateErrorOverrideManager`
- hyper-rustls takes care of setting ALPN protocols for HTTP requests,
  so for WebSockets this is moved to the WebSocket code.
- The safe set of cypher suites is chosen, which seem to correspond to
  the "Modern" configuration from [1]. This can be adjusted later.
- Instead of passing a string of PEM CA certificates around, an enum is
  used that includes parsed Certificates (or the default which reads
  them from webpki-roots).
- Code for starting up an SSL server for testing is cleaned up a little,
  due to the fact that the certificates need to be overriden explicitly
  now. This is due to the fact that the `webpki` crate is more stringent
  with self-signed certificates than SSL (CA certificates cannot used as
  end-entity certificates). [2]

1. https://wiki.mozilla.org/Security/Server_Side_TLS
2. https://github.com/briansmith/webpki/issues/114

Fixes #7888.
Fixes #13749.
Fixes #26835.
Fixes #29291.
This commit is contained in:
Martin Robinson 2023-08-08 16:00:10 +02:00 committed by GitHub
parent ab0f48f8e8
commit bce7622cde
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
31 changed files with 575 additions and 4399 deletions

View file

@ -6,7 +6,6 @@ license = "MPL-2.0"
edition = "2018"
publish = false
autotests = false # Inhibit lookup for tests/*.rs without [[test]] sections
build = "build.rs"
[lib]
name = "net"
@ -16,7 +15,7 @@ doctest = false
[dependencies]
async-recursion = "0.3.2"
async-tungstenite = { version = "0.22", features = ["tokio-openssl"] }
async-tungstenite = { workspace = true }
base64 = { workspace = true }
brotli = "3"
bytes = "1"
@ -33,7 +32,7 @@ generic-array = "0.14"
headers = { workspace = true }
http = { workspace = true }
hyper = { workspace = true, features = ["client", "http1", "http2", "tcp", "stream"] }
hyper-openssl = "0.9.1"
hyper-rustls = { workspace = true }
hyper_serde = { workspace = true }
immeta = "0.4"
ipc-channel = { workspace = true }
@ -46,12 +45,12 @@ mime = { workspace = true }
mime_guess = { workspace = true }
msg = { path = "../msg" }
net_traits = { path = "../net_traits" }
openssl = "0.10"
openssl-sys = "0.9"
percent-encoding = { workspace = true }
pixels = { path = "../pixels" }
profile_traits = { path = "../profile_traits" }
rayon = { workspace = true }
rustls = { workspace = true }
rustls-pemfile = { workspace = true }
serde = { workspace = true }
serde_json = { workspace = true }
servo_allocator = { path = "../allocator" }
@ -61,16 +60,17 @@ servo_url = { path = "../url" }
sha2 = "0.10"
time = { workspace = true }
tokio = { workspace = true, features = ["sync", "macros", "rt-multi-thread"] }
tokio-rustls = { workspace = true }
tokio-stream = "0.1"
tungstenite = "0.19"
tungstenite = { workspace = true }
url = { workspace = true }
uuid = { workspace = true }
webrender_api = { workspace = true }
webpki-roots = { workspace = true }
[dev-dependencies]
futures = {version = "0.3", features = ["compat"]}
std_test_override = { path = "../std_test_override" }
tokio-openssl = "0.6"
tokio-test = "0.4"
tokio-stream = { version = "0.1", features = ["net"] }
hyper = { workspace = true, features = ["full"] }

View file

@ -1,16 +0,0 @@
/* This Source Code Form is subject to the terms of the Mozilla Public
* License, v. 2.0. If a copy of the MPL was not distributed with this
* file, You can obtain one at https://mozilla.org/MPL/2.0/. */
fn main() {
let version =
std::env::var("DEP_OPENSSL_VERSION_NUMBER").expect("missing DEP_OPENSSL_VERSION_NUMBER");
let actual = u64::from_str_radix(&version, 16).unwrap();
let minimum = 0x10101000;
if actual < minimum {
panic!(
"Your OpenSSL version is older than 1.1.1 ({:x}), you have: {:x}",
minimum, actual
);
}
}

View file

@ -9,80 +9,30 @@ use http::uri::{Authority, Uri as Destination};
use hyper::client::HttpConnector as HyperHttpConnector;
use hyper::rt::Executor;
use hyper::{service::Service, Body, Client};
use hyper_openssl::HttpsConnector;
use openssl::ex_data::Index;
use openssl::ssl::{
Ssl, SslConnector, SslConnectorBuilder, SslContext, SslMethod, SslOptions, SslVerifyMode,
};
use openssl::x509::{self, X509StoreContext};
use std::collections::hash_map::{Entry, HashMap};
use hyper_rustls::HttpsConnector as HyperRustlsHttpsConnector;
use rustls::client::WebPkiVerifier;
use rustls::{Certificate, ClientConfig, OwnedTrustAnchor, RootCertStore, ServerName};
use std::collections::hash_map::HashMap;
use std::convert::TryFrom;
use std::sync::{Arc, Mutex};
pub const BUF_SIZE: usize = 32768;
pub const ALPN_H2_H1: &'static [u8] = b"\x02h2\x08http/1.1";
pub const ALPN_H1: &'static [u8] = b"\x08http/1.1";
// See https://wiki.mozilla.org/Security/Server_Side_TLS for orientation.
const TLS1_2_CIPHERSUITES: &'static str = concat!(
"ECDHE-ECDSA-CHACHA20-POLY1305:ECDHE-RSA-CHACHA20-POLY1305:",
"ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-ECDSA-AES128-GCM-SHA256:",
"ECDHE-RSA-AES256-GCM-SHA384:ECDHE-RSA-AES128-GCM-SHA256:",
"ECDHE-RSA-AES256-SHA:ECDHE-RSA-AES128-SHA@SECLEVEL=2"
);
const SIGNATURE_ALGORITHMS: &'static str = concat!(
"ed448:ed25519:",
"ECDSA+SHA384:ECDSA+SHA256:",
"RSA-PSS+SHA512:RSA-PSS+SHA384:RSA-PSS+SHA256:",
"RSA+SHA512:RSA+SHA384:RSA+SHA256"
);
#[derive(Clone)]
pub struct ConnectionCerts {
certs: Arc<Mutex<HashMap<String, (Vec<u8>, u32)>>>,
}
impl ConnectionCerts {
pub fn new() -> Self {
Self {
certs: Arc::new(Mutex::new(HashMap::new())),
}
}
fn store(&self, host: String, cert_bytes: Vec<u8>) {
let mut certs = self.certs.lock().unwrap();
let entry = certs.entry(host).or_insert((cert_bytes, 0));
entry.1 += 1;
}
pub(crate) fn remove(&self, host: String) -> Option<Vec<u8>> {
match self.certs.lock().unwrap().entry(host) {
Entry::Vacant(_) => return None,
Entry::Occupied(mut e) => {
e.get_mut().1 -= 1;
if e.get().1 == 0 {
return Some((e.remove_entry().1).0);
}
Some(e.get().0.clone())
},
}
}
}
#[derive(Clone)]
pub struct HttpConnector {
pub struct ServoHttpConnector {
inner: HyperHttpConnector,
}
impl HttpConnector {
fn new() -> HttpConnector {
impl ServoHttpConnector {
fn new() -> ServoHttpConnector {
let mut inner = HyperHttpConnector::new();
inner.enforce_http(false);
inner.set_happy_eyeballs_timeout(None);
HttpConnector { inner }
ServoHttpConnector { inner }
}
}
impl Service<Destination> for HttpConnector {
impl Service<Destination> for ServoHttpConnector {
type Response = <HyperHttpConnector as Service<Destination>>::Response;
type Error = <HyperHttpConnector as Service<Destination>>::Error;
type Future = <HyperHttpConnector as Service<Destination>>::Future;
@ -118,119 +68,85 @@ impl Service<Destination> for HttpConnector {
}
}
pub type Connector = HttpsConnector<HttpConnector>;
pub type TlsConfig = SslConnectorBuilder;
pub type Connector = HyperRustlsHttpsConnector<ServoHttpConnector>;
pub type TlsConfig = ClientConfig;
#[derive(Clone)]
pub struct ExtraCerts(Arc<Mutex<Vec<Vec<u8>>>>);
#[derive(Clone, Debug, Default)]
struct CertificateErrorOverrideManagerInternal {
/// A mapping of certificates and their hosts, which have seen certificate errors.
/// This is used to later create an override in this [CertificateErrorOverrideManager].
certificates_failing_to_verify: HashMap<ServerName, Certificate>,
/// A list of certificates that should be accepted despite encountering verification
/// errors.
overrides: Vec<Certificate>,
}
impl ExtraCerts {
/// This data structure is used to track certificate verification errors and overrides.
/// It tracks:
/// - A list of [Certificate]s with verification errors mapped by their [ServerName]
/// - A list of [Certificate]s for which to ignore verification errors.
#[derive(Clone, Debug, Default)]
pub struct CertificateErrorOverrideManager(Arc<Mutex<CertificateErrorOverrideManagerInternal>>);
impl CertificateErrorOverrideManager {
pub fn new() -> Self {
Self(Arc::new(Mutex::new(vec![])))
Self(Default::default())
}
pub fn add(&self, bytes: Vec<u8>) {
self.0.lock().unwrap().push(bytes);
/// Add a certificate to this manager's list of certificates for which to ignore
/// validation errors.
pub fn add_override(&self, certificate: &Certificate) {
self.0.lock().unwrap().overrides.push(certificate.clone());
}
/// Given the a string representation of a sever host name, remove information about
/// a [Certificate] with verification errors. If a certificate with
/// verification errors was found, return it, otherwise None.
pub(crate) fn remove_certificate_failing_verification(
&self,
host: &str,
) -> Option<Certificate> {
let server_name = match ServerName::try_from(host) {
Ok(name) => name,
Err(error) => {
warn!("Could not convert host string into RustTLS ServerName: {error:?}");
return None;
},
};
self.0
.lock()
.unwrap()
.certificates_failing_to_verify
.remove(&server_name)
}
}
struct Host(String);
lazy_static! {
static ref EXTRA_INDEX: Index<SslContext, ExtraCerts> = SslContext::new_ex_index().unwrap();
static ref CONNECTION_INDEX: Index<SslContext, ConnectionCerts> =
SslContext::new_ex_index().unwrap();
static ref HOST_INDEX: Index<Ssl, Host> = Ssl::new_ex_index().unwrap();
#[derive(Clone, Debug)]
pub enum CACertificates {
Default,
Override(RootCertStore),
}
/// Create a [TlsConfig] to use for managing a HTTP connection. This currently creates
/// a rustls [ClientConfig].
///
/// FIXME: The `ignore_certificate_errors` argument ignores all certificate errors. This
/// is used when running the WPT tests, because rustls currently rejects the WPT certificiate.
/// See https://github.com/servo/servo/issues/30080
pub fn create_tls_config(
certs: &str,
alpn: &[u8],
extra_certs: ExtraCerts,
connection_certs: ConnectionCerts,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
override_manager: CertificateErrorOverrideManager,
) -> TlsConfig {
// certs include multiple certificates. We could add all of them at once,
// but if any of them were already added, openssl would fail to insert all
// of them.
let mut certs = certs;
let mut cfg = SslConnector::builder(SslMethod::tls()).unwrap();
loop {
let token = "-----END CERTIFICATE-----";
if let Some(index) = certs.find(token) {
let (cert, rest) = certs.split_at(index + token.len());
certs = rest;
let cert = x509::X509::from_pem(cert.as_bytes()).unwrap();
cfg.cert_store_mut()
.add_cert(cert)
.or_else(|e| {
let v: Option<Option<&str>> = e.errors().iter().nth(0).map(|e| e.reason());
if v == Some(Some("cert already in hash table")) {
warn!("Cert already in hash table. Ignoring.");
// Ignore error X509_R_CERT_ALREADY_IN_HASH_TABLE which means the
// certificate is already in the store.
Ok(())
} else {
Err(e)
}
})
.expect("could not set CA file");
} else {
break;
}
}
cfg.set_alpn_protos(alpn)
.expect("could not set alpn protocols");
cfg.set_cipher_list(TLS1_2_CIPHERSUITES)
.expect("could not set TLS 1.2 ciphersuites");
cfg.set_sigalgs_list(SIGNATURE_ALGORITHMS)
.expect("could not set signature algorithms");
cfg.set_options(
SslOptions::NO_SSLV2 |
SslOptions::NO_SSLV3 |
SslOptions::NO_TLSV1 |
SslOptions::NO_TLSV1_1 |
SslOptions::NO_COMPRESSION,
let verifier = CertificateVerificationOverrideVerifier::new(
ca_certificates,
ignore_certificate_errors,
override_manager,
);
cfg.set_ex_data(*EXTRA_INDEX, extra_certs);
cfg.set_ex_data(*CONNECTION_INDEX, connection_certs);
cfg.set_verify_callback(SslVerifyMode::PEER, |verified, x509_store_context| {
if verified {
return true;
}
let ssl_idx = X509StoreContext::ssl_idx().unwrap();
let ssl = x509_store_context.ex_data(ssl_idx).unwrap();
// Obtain the cert bytes for this connection.
let cert = match x509_store_context.current_cert() {
Some(cert) => cert,
None => return false,
};
let pem = match cert.to_pem() {
Ok(pem) => pem,
Err(_) => return false,
};
let ssl_context = ssl.ssl_context();
// Ensure there's an entry stored in the set of known connection certs for this connection.
if let Some(host) = ssl.ex_data(*HOST_INDEX) {
let connection_certs = ssl_context.ex_data(*CONNECTION_INDEX).unwrap();
connection_certs.store((*host).0.clone(), pem.clone());
}
// Fall back to the dynamic set of allowed certs.
let extra_certs = ssl_context.ex_data(*EXTRA_INDEX).unwrap();
for cert in &*extra_certs.0.lock().unwrap() {
if pem == *cert {
return true;
}
}
false
});
cfg
rustls::ClientConfig::builder()
.with_safe_defaults()
.with_custom_certificate_verifier(Arc::new(verifier))
.with_no_client_auth()
}
struct TokioExecutor {}
@ -244,14 +160,95 @@ where
}
}
pub fn create_http_client(tls_config: TlsConfig) -> Client<Connector, Body> {
let mut connector = HttpsConnector::with_connector(HttpConnector::new(), tls_config).unwrap();
connector.set_callback(|configuration, destination| {
if let Some(host) = destination.host() {
configuration.set_ex_data(*HOST_INDEX, Host(host.to_owned()));
struct CertificateVerificationOverrideVerifier {
webpki_verifier: WebPkiVerifier,
ignore_certificate_errors: bool,
override_manager: CertificateErrorOverrideManager,
}
impl CertificateVerificationOverrideVerifier {
fn new(
ca_certficates: CACertificates,
ignore_certificate_errors: bool,
override_manager: CertificateErrorOverrideManager,
) -> Self {
let root_cert_store = match ca_certficates {
CACertificates::Default => {
let mut root_cert_store = rustls::RootCertStore::empty();
root_cert_store.add_trust_anchors(webpki_roots::TLS_SERVER_ROOTS.0.iter().map(
|trust_anchor| {
OwnedTrustAnchor::from_subject_spki_name_constraints(
trust_anchor.subject,
trust_anchor.spki,
trust_anchor.name_constraints,
)
},
));
root_cert_store
},
CACertificates::Override(root_cert_store) => root_cert_store,
};
Self {
// See https://github.com/rustls/rustls/blame/v/0.21.6/rustls/src/client/builder.rs#L141
// This is the default verifier for Rustls that we are wrapping.
webpki_verifier: WebPkiVerifier::new(root_cert_store, None),
ignore_certificate_errors,
override_manager,
}
Ok(())
});
}
}
impl rustls::client::ServerCertVerifier for CertificateVerificationOverrideVerifier {
fn verify_server_cert(
&self,
end_entity: &Certificate,
intermediates: &[Certificate],
server_name: &ServerName,
scts: &mut dyn Iterator<Item = &[u8]>,
ocsp_response: &[u8],
now: std::time::SystemTime,
) -> Result<rustls::client::ServerCertVerified, rustls::Error> {
let error = match self.webpki_verifier.verify_server_cert(
end_entity,
intermediates,
server_name,
scts,
ocsp_response,
now,
) {
Ok(result) => return Ok(result),
Err(error) => error,
};
if self.ignore_certificate_errors {
warn!("Ignoring certficate error: {error:?}");
return Ok(rustls::client::ServerCertVerified::assertion());
}
// If there's an override for this certificate, just accept it.
for cert_with_exception in &*self.override_manager.0.lock().unwrap().overrides {
if *end_entity == *cert_with_exception {
return Ok(rustls::client::ServerCertVerified::assertion());
}
}
self.override_manager
.0
.lock()
.unwrap()
.certificates_failing_to_verify
.insert(server_name.clone(), end_entity.clone());
Err(error)
}
}
pub fn create_http_client(tls_config: TlsConfig) -> Client<Connector, Body> {
let connector = hyper_rustls::HttpsConnectorBuilder::new()
.with_tls_config(tls_config)
.https_or_http()
.enable_http1()
.enable_http2()
.wrap_connector(ServoHttpConnector::new());
Client::builder()
.http1_title_case_headers(true)

View file

@ -9,7 +9,7 @@ use crate::filemanager_thread::{FileManager, FILE_CHUNK_SIZE};
use crate::http_loader::{determine_requests_referrer, http_fetch, HttpState};
use crate::http_loader::{set_default_accept, set_default_accept_language};
use crate::subresource_integrity::is_response_integrity_valid;
use base64::Engine;
use base64::{engine::general_purpose, Engine as _};
use content_security_policy as csp;
use crossbeam_channel::Sender;
use devtools_traits::DevtoolsControlMsg;
@ -31,11 +31,12 @@ use net_traits::request::{
use net_traits::response::{Response, ResponseBody, ResponseType};
use net_traits::{FetchTaskTarget, NetworkError, ReferrerPolicy, ResourceFetchTiming};
use net_traits::{ResourceAttribute, ResourceTimeValue, ResourceTimingType};
use rustls::Certificate;
use servo_arc::Arc as ServoArc;
use servo_url::ServoUrl;
use std::borrow::Cow;
use std::fs::File;
use std::io::{BufReader, Seek, SeekFrom};
use std::io::{self, BufReader, Seek, SeekFrom};
use std::mem;
use std::ops::Bound;
use std::str;
@ -627,6 +628,48 @@ fn create_blank_reply(url: ServoUrl, timing_type: ResourceTimingType) -> Respons
response
}
/// Handle a request from the user interface to ignore validation errors for a certificate.
fn handle_allowcert_request(request: &mut Request, context: &FetchContext) -> io::Result<()> {
let error = |string| Err(io::Error::new(io::ErrorKind::Other, string));
let body = match request.body.as_mut() {
Some(body) => body,
None => return error("No body found"),
};
let stream = body.take_stream();
let stream = stream.lock().unwrap();
let (body_chan, body_port) = ipc::channel().unwrap();
let _ = stream.send(BodyChunkRequest::Connect(body_chan));
let _ = stream.send(BodyChunkRequest::Chunk);
let body_bytes = match body_port.recv().ok() {
Some(BodyChunkResponse::Chunk(bytes)) => bytes,
_ => return error("Certificate not sent in a single chunk"),
};
let split_idx = match body_bytes.iter().position(|b| *b == b'&') {
Some(split_idx) => split_idx,
None => return error("Could not find ampersand in data"),
};
let (secret, cert_base64) = body_bytes.split_at(split_idx);
let secret = str::from_utf8(secret).ok().and_then(|s| s.parse().ok());
if secret != Some(*net_traits::PRIVILEGED_SECRET) {
return error("Invalid secret sent. Ignoring request");
}
let cert_bytes = match general_purpose::STANDARD_NO_PAD.decode(&cert_base64[1..]) {
Ok(bytes) => bytes,
Err(_) => return error("Could not decode certificate base64"),
};
context
.state
.override_manager
.add_override(&Certificate(cert_bytes));
Ok(())
}
/// [Scheme fetch](https://fetch.spec.whatwg.org#scheme-fetch)
async fn scheme_fetch(
request: &mut Request,
@ -641,32 +684,9 @@ async fn scheme_fetch(
"about" if url.path() == "blank" => create_blank_reply(url, request.timing_type()),
"chrome" if url.path() == "allowcert" => {
let data = request.body.as_mut().and_then(|body| {
let stream = body.take_stream();
let stream = stream.lock().unwrap();
let (body_chan, body_port) = ipc::channel().unwrap();
let _ = stream.send(BodyChunkRequest::Connect(body_chan));
let _ = stream.send(BodyChunkRequest::Chunk);
match body_port.recv().ok() {
Some(BodyChunkResponse::Chunk(bytes)) => Some(bytes),
_ => panic!("cert should be sent in a single chunk."),
}
});
let data = data.as_ref().and_then(|b| {
let idx = b.iter().position(|b| *b == b'&')?;
Some(b.split_at(idx))
});
if let Some((secret, bytes)) = data {
let secret = str::from_utf8(secret).ok().and_then(|s| s.parse().ok());
if secret == Some(*net_traits::PRIVILEGED_SECRET) {
if let Ok(bytes) = base64::engine::general_purpose::STANDARD.decode(&bytes[1..])
{
context.state.extra_certs.add(bytes);
}
}
if let Err(error) = handle_allowcert_request(request, context) {
warn!("Could not handle allowcert request: {error}");
}
create_blank_reply(url, request.timing_type())
},

View file

@ -2,7 +2,10 @@
* License, v. 2.0. If a copy of the MPL was not distributed with this
* file, You can obtain one at https://mozilla.org/MPL/2.0/. */
use crate::connector::{create_http_client, ConnectionCerts, Connector, ExtraCerts, TlsConfig};
use crate::connector::{
create_http_client, create_tls_config, CACertificates, CertificateErrorOverrideManager,
Connector,
};
use crate::cookie;
use crate::cookie_storage::CookieStorage;
use crate::decoder::Decoder;
@ -98,12 +101,12 @@ pub struct HttpState {
pub auth_cache: RwLock<AuthCache>,
pub history_states: RwLock<HashMap<HistoryStateId, Vec<u8>>>,
pub client: Client<Connector, Body>,
pub extra_certs: ExtraCerts,
pub connection_certs: ConnectionCerts,
pub override_manager: CertificateErrorOverrideManager,
}
impl HttpState {
pub fn new(tls_config: TlsConfig) -> HttpState {
pub fn new() -> HttpState {
let override_manager = CertificateErrorOverrideManager::new();
HttpState {
hsts_list: RwLock::new(HstsList::new()),
cookie_jar: RwLock::new(CookieStorage::new(150)),
@ -111,9 +114,12 @@ impl HttpState {
history_states: RwLock::new(HashMap::new()),
http_cache: RwLock::new(HttpCache::new()),
http_cache_state: Mutex::new(HashMap::new()),
client: create_http_client(tls_config),
extra_certs: ExtraCerts::new(),
connection_certs: ConnectionCerts::new(),
client: create_http_client(create_tls_config(
CACertificates::Default,
false, /* ignore_certificate_errors */
override_manager.clone(),
)),
override_manager,
}
}
}
@ -652,18 +658,12 @@ async fn obtain_response(
let send_start = precise_time_ms();
let host = request.uri().host().unwrap_or("").to_owned();
let host_clone = request.uri().host().unwrap_or("").to_owned();
let connection_certs = context.state.connection_certs.clone();
let connection_certs_clone = context.state.connection_certs.clone();
let override_manager = context.state.override_manager.clone();
let headers = headers.clone();
client
.request(request)
.and_then(move |res| {
// We no longer need to track the cert for this connection.
connection_certs.remove(host);
let send_end = precise_time_ms();
// TODO(#21271) response_start: immediately after receiving first byte of response
@ -696,8 +696,11 @@ async fn obtain_response(
};
future::ready(Ok((Decoder::detect(res), msg)))
})
.map_err(move |e| {
NetworkError::from_hyper_error(&e, connection_certs_clone.remove(host_clone))
.map_err(move |error| {
NetworkError::from_hyper_error(
&error,
override_manager.remove_certificate_failing_verification(host.as_str()),
)
})
.await
}

View file

@ -46,6 +46,3 @@ pub mod test {
pub use crate::hosts::{parse_hostsfile, replace_host_table};
pub use crate::http_loader::HttpState;
}
// This dependency gives `build.rs` access to the `DEP_OPENSSL_VERSION_NUMBER` env variable.
use openssl_sys as _;

View file

@ -5,7 +5,7 @@
//! A thread that takes a URL and streams back the binary data.
use crate::connector::{
create_http_client, create_tls_config, ConnectionCerts, ExtraCerts, ALPN_H2_H1,
create_http_client, create_tls_config, CACertificates, CertificateErrorOverrideManager,
};
use crate::cookie;
use crate::cookie_storage::CookieStorage;
@ -19,7 +19,6 @@ use crate::storage_thread::StorageThreadFactory;
use crate::websocket_loader;
use crossbeam_channel::Sender;
use devtools_traits::DevtoolsControlMsg;
use embedder_traits::resources::{self, Resource};
use embedder_traits::EmbedderProxy;
use hyper_serde::Serde;
use ipc_channel::ipc::{self, IpcReceiver, IpcReceiverSet, IpcSender};
@ -39,19 +38,30 @@ use net_traits::{ResourceThreads, WebSocketDomAction};
use profile_traits::mem::ProfilerChan as MemProfilerChan;
use profile_traits::mem::{Report, ReportKind, ReportsChan};
use profile_traits::time::ProfilerChan;
use rustls::RootCertStore;
use serde::{Deserialize, Serialize};
use servo_arc::Arc as ServoArc;
use servo_url::{ImmutableOrigin, ServoUrl};
use std::borrow::{Cow, ToOwned};
use std::collections::HashMap;
use std::fs::{self, File};
use std::io::prelude::*;
use std::fs::File;
use std::io::{self, prelude::*, BufReader};
use std::ops::Deref;
use std::path::{Path, PathBuf};
use std::sync::{Arc, Mutex, RwLock};
use std::thread;
use std::time::Duration;
/// Load a file with CA certificate and produce a RootCertStore with the results.
fn load_root_cert_store_from_file(file_path: String) -> io::Result<RootCertStore> {
let mut root_cert_store = RootCertStore::empty();
let mut pem = BufReader::new(File::open(file_path)?);
let certs = rustls_pemfile::certs(&mut pem)?;
root_cert_store.add_parsable_certificates(&certs);
Ok(root_cert_store)
}
/// Returns a tuple of (public, private) senders to the new threads.
pub fn new_resource_threads(
user_agent: Cow<'static, str>,
@ -61,7 +71,19 @@ pub fn new_resource_threads(
embedder_proxy: EmbedderProxy,
config_dir: Option<PathBuf>,
certificate_path: Option<String>,
ignore_certificate_errors: bool,
) -> (ResourceThreads, ResourceThreads) {
let ca_certificates = match certificate_path {
Some(path) => match load_root_cert_store_from_file(path) {
Ok(root_cert_store) => CACertificates::Override(root_cert_store),
Err(error) => {
warn!("Could not load CA file. Falling back to defaults. {error:?}");
CACertificates::Default
},
},
None => CACertificates::Default,
};
let (public_core, private_core) = new_core_resource_thread(
user_agent,
devtools_sender,
@ -69,7 +91,8 @@ pub fn new_resource_threads(
mem_profiler_chan,
embedder_proxy,
config_dir.clone(),
certificate_path,
ca_certificates,
ignore_certificate_errors,
);
let storage: IpcSender<StorageThreadMsg> = StorageThreadFactory::new(config_dir);
(
@ -86,7 +109,8 @@ pub fn new_core_resource_thread(
mem_profiler_chan: MemProfilerChan,
embedder_proxy: EmbedderProxy,
config_dir: Option<PathBuf>,
certificate_path: Option<String>,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
) -> (CoreResourceThread, CoreResourceThread) {
let (public_setup_chan, public_setup_port) = ipc::channel().unwrap();
let (private_setup_chan, private_setup_port) = ipc::channel().unwrap();
@ -100,13 +124,15 @@ pub fn new_core_resource_thread(
devtools_sender,
time_profiler_chan,
embedder_proxy,
certificate_path.clone(),
ca_certificates.clone(),
ignore_certificate_errors,
);
let mut channel_manager = ResourceChannelManager {
resource_manager,
config_dir,
certificate_path,
ca_certificates,
ignore_certificate_errors,
};
mem_profiler_chan.run_with_memory_reporting(
@ -123,12 +149,14 @@ pub fn new_core_resource_thread(
struct ResourceChannelManager {
resource_manager: CoreResourceManager,
config_dir: Option<PathBuf>,
certificate_path: Option<String>,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
}
fn create_http_states(
config_dir: Option<&Path>,
certificate_path: Option<String>,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
) -> (Arc<HttpState>, Arc<HttpState>) {
let mut hsts_list = HstsList::from_servo_preload();
let mut auth_cache = AuthCache::new();
@ -140,14 +168,7 @@ fn create_http_states(
read_json_from_file(&mut cookie_jar, config_dir, "cookie_jar.json");
}
let certs = match certificate_path {
Some(ref path) => fs::read_to_string(path).expect("Couldn't not find certificate file"),
None => resources::read_string(Resource::SSLCertificates),
};
let extra_certs = ExtraCerts::new();
let connection_certs = ConnectionCerts::new();
let override_manager = CertificateErrorOverrideManager::new();
let http_state = HttpState {
hsts_list: RwLock::new(hsts_list),
cookie_jar: RwLock::new(cookie_jar),
@ -156,18 +177,14 @@ fn create_http_states(
http_cache: RwLock::new(http_cache),
http_cache_state: Mutex::new(HashMap::new()),
client: create_http_client(create_tls_config(
&certs,
ALPN_H2_H1,
extra_certs.clone(),
connection_certs.clone(),
ca_certificates.clone(),
ignore_certificate_errors,
override_manager.clone(),
)),
extra_certs,
connection_certs,
override_manager,
};
let extra_certs = ExtraCerts::new();
let connection_certs = ConnectionCerts::new();
let override_manager = CertificateErrorOverrideManager::new();
let private_http_state = HttpState {
hsts_list: RwLock::new(HstsList::from_servo_preload()),
cookie_jar: RwLock::new(CookieStorage::new(150)),
@ -176,13 +193,11 @@ fn create_http_states(
http_cache: RwLock::new(HttpCache::new()),
http_cache_state: Mutex::new(HashMap::new()),
client: create_http_client(create_tls_config(
&certs,
ALPN_H2_H1,
extra_certs.clone(),
connection_certs.clone(),
ca_certificates,
ignore_certificate_errors,
override_manager.clone(),
)),
extra_certs,
connection_certs,
override_manager,
};
(Arc::new(http_state), Arc::new(private_http_state))
@ -198,7 +213,8 @@ impl ResourceChannelManager {
) {
let (public_http_state, private_http_state) = create_http_states(
self.config_dir.as_ref().map(Deref::deref),
self.certificate_path.clone(),
self.ca_certificates.clone(),
self.ignore_certificate_errors,
);
let mut rx_set = IpcReceiverSet::new().unwrap();
@ -455,7 +471,8 @@ pub struct CoreResourceManager {
sw_managers: HashMap<ImmutableOrigin, IpcSender<CustomResponseMediator>>,
filemanager: FileManager,
thread_pool: Arc<CoreResourceThreadPool>,
certificate_path: Option<String>,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
}
/// The state of the thread-pool used by CoreResource.
@ -589,7 +606,8 @@ impl CoreResourceManager {
devtools_sender: Option<Sender<DevtoolsControlMsg>>,
_profiler_chan: ProfilerChan,
embedder_proxy: EmbedderProxy,
certificate_path: Option<String>,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
) -> CoreResourceManager {
let pool = CoreResourceThreadPool::new(16);
let pool_handle = Arc::new(pool);
@ -599,7 +617,8 @@ impl CoreResourceManager {
sw_managers: Default::default(),
filemanager: FileManager::new(embedder_proxy, Arc::downgrade(&pool_handle)),
thread_pool: pool_handle,
certificate_path,
ca_certificates,
ignore_certificate_errors,
}
}
@ -725,7 +744,8 @@ impl CoreResourceManager {
event_sender,
action_receiver,
http_state.clone(),
self.certificate_path.clone(),
self.ca_certificates.clone(),
self.ignore_certificate_errors,
);
}
}

View file

@ -24,7 +24,6 @@ use hyper::Body;
use hyper::{Request as HyperRequest, Response as HyperResponse};
use mime::{self, Mime};
use msg::constellation_msg::TEST_PIPELINE_ID;
use net::connector::{create_tls_config, ConnectionCerts, ExtraCerts, ALPN_H2_H1};
use net::fetch::cors_cache::CorsCache;
use net::fetch::methods::{self, CancellationListener, FetchContext};
use net::filemanager_thread::FileManager;
@ -756,24 +755,10 @@ fn test_fetch_with_hsts() {
*response.body_mut() = MESSAGE.to_vec().into();
};
let cert_path = Path::new("../../resources/self_signed_certificate_for_testing.crt")
.canonicalize()
.unwrap();
let key_path = Path::new("../../resources/privatekey_for_testing.key")
.canonicalize()
.unwrap();
let (server, url) = make_ssl_server(handler, cert_path.clone(), key_path.clone());
let certs = fs::read_to_string(cert_path).expect("Couldn't find certificate file");
let tls_config = create_tls_config(
&certs,
ALPN_H2_H1,
ExtraCerts::new(),
ConnectionCerts::new(),
);
let (server, url) = make_ssl_server(handler);
let mut context = FetchContext {
state: Arc::new(HttpState::new(tls_config)),
state: Arc::new(HttpState::new()),
user_agent: DEFAULT_USER_AGENT.into(),
devtools_chan: None,
filemanager: Arc::new(Mutex::new(FileManager::new(
@ -787,6 +772,12 @@ fn test_fetch_with_hsts() {
))),
};
// The server certificate is self-signed, so we need to add an override
// so that the connection works properly.
for certificate in server.certificates.as_ref().unwrap().iter() {
context.state.override_manager.add_override(certificate);
}
{
let mut list = context.state.hsts_list.write().unwrap();
list.push(
@ -821,25 +812,12 @@ fn test_load_adds_host_to_hsts_list_when_url_is_https() {
));
*response.body_mut() = b"Yay!".to_vec().into();
};
let cert_path = Path::new("../../resources/self_signed_certificate_for_testing.crt")
.canonicalize()
.unwrap();
let key_path = Path::new("../../resources/privatekey_for_testing.key")
.canonicalize()
.unwrap();
let (server, mut url) = make_ssl_server(handler, cert_path.clone(), key_path.clone());
let (server, mut url) = make_ssl_server(handler);
url.as_mut_url().set_scheme("https").unwrap();
let certs = fs::read_to_string(cert_path).expect("Couldn't find certificate file");
let tls_config = create_tls_config(
&certs,
ALPN_H2_H1,
ExtraCerts::new(),
ConnectionCerts::new(),
);
let mut context = FetchContext {
state: Arc::new(HttpState::new(tls_config)),
state: Arc::new(HttpState::new()),
user_agent: DEFAULT_USER_AGENT.into(),
devtools_chan: None,
filemanager: Arc::new(Mutex::new(FileManager::new(
@ -853,6 +831,12 @@ fn test_load_adds_host_to_hsts_list_when_url_is_https() {
))),
};
// The server certificate is self-signed, so we need to add an override
// so that the connection works properly.
for certificate in server.certificates.as_ref().unwrap().iter() {
context.state.override_manager.add_override(certificate);
}
let mut request = RequestBuilder::new(url.clone(), Referrer::NoReferrer)
.method(Method::GET)
.body(None)
@ -885,29 +869,12 @@ fn test_fetch_self_signed() {
let handler = move |_: HyperRequest<Body>, response: &mut HyperResponse<Body>| {
*response.body_mut() = b"Yay!".to_vec().into();
};
let client_cert_path = Path::new("../../resources/certs").canonicalize().unwrap();
let cert_path = Path::new("../../resources/self_signed_certificate_for_testing.crt")
.canonicalize()
.unwrap();
let key_path = Path::new("../../resources/privatekey_for_testing.key")
.canonicalize()
.unwrap();
let (_server, mut url) = make_ssl_server(handler, cert_path.clone(), key_path.clone());
let (server, mut url) = make_ssl_server(handler);
url.as_mut_url().set_scheme("https").unwrap();
let cert_data = fs::read_to_string(cert_path.clone()).expect("Couldn't find certificate file");
let client_cert_data =
fs::read_to_string(client_cert_path.clone()).expect("Couldn't find certificate file");
let extra_certs = ExtraCerts::new();
let tls_config = create_tls_config(
&client_cert_data,
ALPN_H2_H1,
extra_certs.clone(),
ConnectionCerts::new(),
);
let mut context = FetchContext {
state: Arc::new(HttpState::new(tls_config)),
state: Arc::new(HttpState::new()),
user_agent: DEFAULT_USER_AGENT.into(),
devtools_chan: None,
filemanager: Arc::new(Mutex::new(FileManager::new(
@ -936,16 +903,11 @@ fn test_fetch_self_signed() {
Some(NetworkError::SslValidation(..))
));
extra_certs.add(cert_data.as_bytes().into());
// FIXME: something weird happens inside the SSL server after the first
// connection encounters a verification error, and it no longer
// accepts new connections that should work fine. We are forced
// to start a new server and connect to that to verfiy that
// the self-signed cert is now accepted.
let (server, mut url) = make_ssl_server(handler, cert_path.clone(), key_path.clone());
url.as_mut_url().set_scheme("https").unwrap();
// The server certificate is self-signed, so we need to add an override
// so that the connection works properly.
for certificate in server.certificates.as_ref().unwrap().iter() {
context.state.override_manager.add_override(certificate);
}
let mut request = RequestBuilder::new(url.clone(), Referrer::NoReferrer)
.method(Method::GET)

View file

@ -22,10 +22,8 @@ mod resource_thread;
mod subresource_integrity;
use core::convert::Infallible;
use core::pin::Pin;
use crossbeam_channel::{unbounded, Sender};
use devtools_traits::DevtoolsControlMsg;
use embedder_traits::resources::{self, Resource};
use embedder_traits::{EmbedderProxy, EventLoopWaker};
use futures::future::ready;
use futures::StreamExt;
@ -33,7 +31,6 @@ use hyper::server::conn::Http;
use hyper::server::Server as HyperServer;
use hyper::service::{make_service_fn, service_fn};
use hyper::{Body, Request as HyperRequest, Response as HyperResponse};
use net::connector::{create_tls_config, ConnectionCerts, ExtraCerts, ALPN_H2_H1};
use net::fetch::cors_cache::CorsCache;
use net::fetch::methods::{self, CancellationListener, FetchContext};
use net::filemanager_thread::FileManager;
@ -43,16 +40,19 @@ use net_traits::filemanager_thread::FileTokenCheck;
use net_traits::request::Request;
use net_traits::response::Response;
use net_traits::{FetchTaskTarget, ResourceFetchTiming, ResourceTimingType};
use openssl::ssl::{Ssl, SslAcceptor, SslFiletype, SslMethod};
use rustls::{self, Certificate, PrivateKey};
use rustls_pemfile::{certs, pkcs8_private_keys};
use servo_arc::Arc as ServoArc;
use servo_url::ServoUrl;
use std::fs::File;
use std::io::{self, BufReader};
use std::net::TcpListener as StdTcpListener;
use std::path::PathBuf;
use std::path::{Path, PathBuf};
use std::sync::{Arc, Mutex, Weak};
use tokio::net::TcpListener;
use tokio::net::TcpStream;
use tokio::runtime::{Builder, Runtime};
use tokio_openssl::SslStream;
use tokio_rustls::{self, TlsAcceptor};
use tokio_stream::wrappers::TcpListenerStream;
use tokio_test::block_on;
@ -102,17 +102,10 @@ fn new_fetch_context(
fc: Option<EmbedderProxy>,
pool_handle: Option<Weak<CoreResourceThreadPool>>,
) -> FetchContext {
let certs = resources::read_string(Resource::SSLCertificates);
let tls_config = create_tls_config(
&certs,
ALPN_H2_H1,
ExtraCerts::new(),
ConnectionCerts::new(),
);
let sender = fc.unwrap_or_else(|| create_embedder_proxy());
FetchContext {
state: Arc::new(HttpState::new(tls_config)),
state: Arc::new(HttpState::new()),
user_agent: DEFAULT_USER_AGENT.into(),
devtools_chan: dc.map(|dc| Arc::new(Mutex::new(dc))),
filemanager: Arc::new(Mutex::new(FileManager::new(
@ -167,6 +160,7 @@ fn fetch_with_cors_cache(request: &mut Request, cache: &mut CorsCache) -> Respon
pub(crate) struct Server {
pub close_channel: tokio::sync::oneshot::Sender<()>,
pub certificates: Option<Vec<Certificate>>,
}
impl Server {
@ -205,11 +199,39 @@ where
};
HANDLE.lock().unwrap().spawn(server);
let server = Server { close_channel: tx };
(server, url)
(
Server {
close_channel: tx,
certificates: None,
},
url,
)
}
fn make_ssl_server<H>(handler: H, cert_path: PathBuf, key_path: PathBuf) -> (Server, ServoUrl)
/// Given a path to a file containing PEM certificates, load and parse them into
/// a vector of RusTLS [Certificate]s.
fn load_certificates_from_pem(path: &PathBuf) -> std::io::Result<Vec<Certificate>> {
let file = File::open(path)?;
let mut reader = BufReader::new(file);
let certs = certs(&mut reader)?;
Ok(certs.into_iter().map(Certificate).collect())
}
/// Given a path to a file containing PEM keys, load and parse them into
/// a vector of RusTLS [PrivateKey]s.
fn load_private_key_from_file(path: &PathBuf) -> Result<PrivateKey, Box<dyn std::error::Error>> {
let file = File::open(&path)?;
let mut reader = BufReader::new(file);
let mut keys = pkcs8_private_keys(&mut reader)?;
match keys.len() {
0 => Err(format!("No PKCS8-encoded private key found in {path:?}").into()),
1 => Ok(PrivateKey(keys.remove(0))),
_ => Err(format!("More than one PKCS8-encoded private key found in {path:?}").into()),
}
}
fn make_ssl_server<H>(handler: H) -> (Server, ServoUrl)
where
H: Fn(HyperRequest<Body>, &mut HyperResponse<Body>) + Send + Sync + 'static,
{
@ -219,13 +241,28 @@ where
.lock()
.unwrap()
.block_on(async move { TcpListener::from_std(listener).unwrap() });
let url_string = format!("http://localhost:{}", listener.local_addr().unwrap().port());
let mut listener = TcpListenerStream::new(listener);
let url = ServoUrl::parse(&url_string).unwrap();
let (tx, mut rx) = tokio::sync::oneshot::channel::<()>();
let cert_path = Path::new("../../resources/self_signed_certificate_for_testing.crt")
.canonicalize()
.unwrap();
let key_path = Path::new("../../resources/privatekey_for_testing.key")
.canonicalize()
.unwrap();
let certificates = load_certificates_from_pem(&cert_path).expect("Invalid certificate");
let key = load_private_key_from_file(&key_path).expect("Invalid key");
let config = rustls::ServerConfig::builder()
.with_safe_defaults()
.with_no_client_auth()
.with_single_cert(certificates.clone(), key)
.map_err(|err| io::Error::new(io::ErrorKind::InvalidInput, err))
.expect("Could not create rustls ServerConfig");
let acceptor = TlsAcceptor::from(Arc::new(config));
let mut listener = TcpListenerStream::new(listener);
let (tx, mut rx) = tokio::sync::oneshot::channel::<()>();
let server = async move {
loop {
let stream = tokio::select! {
@ -244,22 +281,16 @@ where
.unwrap();
let stream = TcpStream::from_std(stream).unwrap();
let mut tls_server_config =
SslAcceptor::mozilla_intermediate_v5(SslMethod::tls()).unwrap();
tls_server_config
.set_certificate_file(&cert_path, SslFiletype::PEM)
.unwrap();
tls_server_config
.set_private_key_file(&key_path, SslFiletype::PEM)
.unwrap();
let tls_server_config = tls_server_config.build();
let ssl = Ssl::new(tls_server_config.context()).unwrap();
let mut stream = SslStream::new(ssl, stream).unwrap();
let _ = Pin::new(&mut stream).accept().await;
let handler = handler.clone();
let acceptor = acceptor.clone();
let stream = match acceptor.accept(stream).await {
Ok(stream) => stream,
Err(_) => {
eprintln!("Error handling TLS stream.");
continue;
},
};
let _ = Http::new()
.serve_connection(
@ -276,6 +307,11 @@ where
HANDLE.lock().unwrap().spawn(server);
let server = Server { close_channel: tx };
(server, url)
(
Server {
close_channel: tx,
certificates: Some(certificates),
},
url,
)
}

View file

@ -4,6 +4,7 @@
use crate::create_embedder_proxy;
use ipc_channel::ipc;
use net::connector::CACertificates;
use net::resource_thread::new_core_resource_thread;
use net::test::parse_hostsfile;
use net_traits::CoreResourceMsg;
@ -27,7 +28,8 @@ fn test_exit() {
MemProfilerChan(mtx),
create_embedder_proxy(),
None,
None,
CACertificates::Default,
false, /* ignore_certificate_errors */
);
resource_thread.send(CoreResourceMsg::Exit(sender)).unwrap();
receiver.recv().unwrap();

View file

@ -11,7 +11,7 @@
//! over events from the network and events from the DOM, using async/await to avoid
//! the need for a dedicated thread per websocket.
use crate::connector::{create_tls_config, ALPN_H1};
use crate::connector::{create_tls_config, CACertificates, TlsConfig};
use crate::cookie::Cookie;
use crate::fetch::methods::should_be_blocked_due_to_bad_port;
use crate::hosts::replace_host;
@ -19,7 +19,6 @@ use crate::http_loader::HttpState;
use async_tungstenite::tokio::{client_async_tls_with_connector_and_config, ConnectStream};
use async_tungstenite::WebSocketStream;
use base64::Engine;
use embedder_traits::resources::{self, Resource};
use futures::future::TryFutureExt;
use futures::sink::SinkExt;
use futures::stream::StreamExt;
@ -29,15 +28,14 @@ use ipc_channel::router::ROUTER;
use net_traits::request::{RequestBuilder, RequestMode};
use net_traits::{CookieSource, MessageData};
use net_traits::{WebSocketDomAction, WebSocketNetworkEvent};
use openssl::ssl::ConnectConfiguration;
use servo_url::ServoUrl;
use std::fs;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, Mutex};
use tokio::net::TcpStream;
use tokio::runtime::Runtime;
use tokio::select;
use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver};
use tokio_rustls::TlsConnector;
use tungstenite::error::Result as WebSocketResult;
use tungstenite::error::{Error, ProtocolError, UrlError};
use tungstenite::handshake::client::{Request, Response};
@ -302,7 +300,7 @@ async fn start_websocket(
resource_event_sender: IpcSender<WebSocketNetworkEvent>,
protocols: Vec<String>,
client: Request,
tls_config: ConnectConfiguration,
tls_config: TlsConfig,
dom_action_receiver: IpcReceiver<WebSocketDomAction>,
) -> Result<(), Error> {
trace!("starting WS connection to {}", url);
@ -330,8 +328,10 @@ async fn start_websocket(
let try_socket = TcpStream::connect((&*domain.to_string(), port)).await;
let socket = try_socket.map_err(Error::Io)?;
let connector = TlsConnector::from(Arc::new(tls_config));
let (stream, response) =
client_async_tls_with_connector_and_config(client, socket, Some(tls_config), None).await?;
client_async_tls_with_connector_and_config(client, socket, Some(connector), None).await?;
let protocol_in_use = process_ws_response(&http_state, &response, &url, &protocols)?;
@ -357,7 +357,8 @@ fn connect(
resource_event_sender: IpcSender<WebSocketNetworkEvent>,
dom_action_receiver: IpcReceiver<WebSocketDomAction>,
http_state: Arc<HttpState>,
certificate_path: Option<String>,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
) -> Result<(), String> {
let protocols = match req_builder.mode {
RequestMode::WebSocket { protocols } => protocols,
@ -382,11 +383,6 @@ fn connect(
return Err("Port blocked".to_string());
}
let certs = match certificate_path {
Some(ref path) => fs::read_to_string(path).map_err(|e| e.to_string())?,
None => resources::read_string(Resource::SSLCertificates),
};
let client = match create_request(
&req_url,
&req_builder.origin.ascii_serialization(),
@ -397,16 +393,12 @@ fn connect(
Err(e) => return Err(e.to_string()),
};
let tls_config = create_tls_config(
&certs,
ALPN_H1,
http_state.extra_certs.clone(),
http_state.connection_certs.clone(),
let mut tls_config = create_tls_config(
ca_certificates,
ignore_certificate_errors,
http_state.override_manager.clone(),
);
let tls_config = match tls_config.build().configure() {
Ok(c) => c,
Err(e) => return Err(e.to_string()),
};
tls_config.alpn_protocols = vec!["h2".to_string().into(), "http/1.1".to_string().into()];
let resource_event_sender2 = resource_event_sender.clone();
match HANDLE.lock().unwrap().as_mut() {
@ -436,7 +428,8 @@ pub fn init(
resource_event_sender: IpcSender<WebSocketNetworkEvent>,
dom_action_receiver: IpcReceiver<WebSocketDomAction>,
http_state: Arc<HttpState>,
certificate_path: Option<String>,
ca_certificates: CACertificates,
ignore_certificate_errors: bool,
) {
let resource_event_sender2 = resource_event_sender.clone();
if let Err(e) = connect(
@ -444,7 +437,8 @@ pub fn init(
resource_event_sender,
dom_action_receiver,
http_state,
certificate_path,
ca_certificates,
ignore_certificate_errors,
) {
warn!("Error starting websocket: {}", e);
let _ = resource_event_sender2.send(WebSocketNetworkEvent::Fail);