From cf356a7b93dd7f5792b13ad92a3d2205a544fc9f Mon Sep 17 00:00:00 2001 From: Simone Basso Date: Wed, 20 Sep 2023 10:46:09 +0200 Subject: [PATCH 1/7] feat(enginenetx): add configurable HTTPS dialer This commit introduces a configurable HTTPS dialer that we will use for implementing beacons, and possibly also other functionality. We need to perform TCP connect and TLS handshake as part of the same goroutine, because we cannot consider a dialing attempt successful after a successful TCP connect. Due to network interference, the dialing may also fail later during the TLS handshake. To support several possible HTTPS dialing strategies, we need to extend what happens during a LookupHost operation. Generally, one would like to have addresses to dial. Rather, here we have tactics, where a single IP address MAY be included into more than a single tactic, if we need to try different tactics with the same address. Also, tactics include a delay, which is useful to (a) avoid performing all operations in parallel, which is not gentle towards otherwise perfectly functioning networks and (b) give penalty to tactics that utilize circumvention, such that we don't even attempt them unless we need to. In turn, the DNS resolver is extended and now it is a policy for configuring dialing. Basically, the policy observes the IP addresses returned by an underlying resolver and then it will decide which tactics to produce based on that. Note that the policy could also extend the set of returned IP addresses when the domain for which we connect is such that we have known IP addresses in advance for such a domain. The default policy we introduce in this commit behaves as follows: 1. it asks the engine to create 16 goroutines for dialing; 2. it uses the DNS lookup results w/o adding any extra IP addr; 3. it produces a tactic for each IP address where we use the domain as the SNI and we add a 300 millisecond delay to the second tactic, 600 to the third, and so on--which is similar to implementing happy eyeballs. It's also worth noting that, tactics MAY override the TLS handshaker being used (for example, to use uTLS) and, also, because they may use different SNIs, the TLS verification is performed AFTER the TLS handshake. Part of https://github.com/ooni/probe/issues/2531 --- internal/enginenetx/httpsdialer.go | 506 ++++++++++++++++++++++++ internal/enginenetx/httpsdialer_test.go | 352 +++++++++++++++++ 2 files changed, 858 insertions(+) create mode 100644 internal/enginenetx/httpsdialer.go create mode 100644 internal/enginenetx/httpsdialer_test.go diff --git a/internal/enginenetx/httpsdialer.go b/internal/enginenetx/httpsdialer.go new file mode 100644 index 0000000000..0ae9438f68 --- /dev/null +++ b/internal/enginenetx/httpsdialer.go @@ -0,0 +1,506 @@ +package enginenetx + +import ( + "context" + "crypto/tls" + "crypto/x509" + "errors" + "fmt" + "net" + "sync" + "sync/atomic" + "time" + + "github.com/ooni/probe-cli/v3/internal/logx" + "github.com/ooni/probe-cli/v3/internal/model" + "github.com/ooni/probe-cli/v3/internal/netxlite" + "github.com/ooni/probe-cli/v3/internal/runtimex" +) + +// HTTPSDialerPolicy describes the policy used by the [*HTTPSDialer]. +type HTTPSDialerPolicy interface { + // LookupTactics performs a DNS lookup for the given domain using the given resolver and + // returns either a list of tactics for dialing or an error. + LookupTactics(ctx context.Context, domain string, reso model.Resolver) ([]HTTPSDialerTactic, error) + + // Parallelism returns the number of goroutines to create when TLS dialing. The + // [HTTPSDialer] will PANIC if the returned number is less than 1. + Parallelism() int +} + +// HTTPSDialerTactic is a tactic to establish a TLS connection. +type HTTPSDialerTactic interface { + // IPAddr returns the IP address to use. + IPAddr() string + + // InitialDelay returns the initial delay before starting the tactic + // or a non-positive value if you don't need any initial delay. + InitialDelay() time.Duration + + // NewTLSHandshaker creates a new [model.TLSHandshaker] given the + // [*netxlite.Netx] and the [model.Logger] we're using. + NewTLSHandshaker(netx *netxlite.Netx, logger model.Logger) model.TLSHandshaker + + // These callbacks are invoked during the TLS handshake to inform this + // tactic about events that occurred. A tactic SHOULD keep track of which + // addresses, SNIs, etc. work and return them more frequently. + OnStarting() + OnTCPConnectError(err error) + OnTLSHandshakeError(err error) + OnTLSVerifyError(err error) + OnSuccess() + + // SNI returns the SNI to send in the TLS Client Hello. + SNI() string + + // Stringer provides a string representation. + fmt.Stringer + + // VerifyHostname returns the hostname to use when verifying + // the X.509 certificate chain returned by the server. + VerifyHostname() string +} + +// HTTPSDialerNullPolicy is the default "null" policy where we use the default +// resolver provided to LookupTactics and we use the correct SNI. +// +// We say that this is the "null" policy because this is what you would get +// by default if you were not using any policy. +// +// This policy uses an Happy-Eyeballs-like algorithm. Dial attempts are +// staggered by 300 milliseconds and up to sixteen dial attempts could be +// active at the same time. Further dials will run once one of the +// sixteen active concurrent dials have failed to connect. +type HTTPSDialerNullPolicy struct{} + +var _ HTTPSDialerPolicy = &HTTPSDialerNullPolicy{} + +// LookupTactics implements HTTPSDialerPolicy. +func (*HTTPSDialerNullPolicy) LookupTactics( + ctx context.Context, domain string, reso model.Resolver) ([]HTTPSDialerTactic, error) { + addrs, err := reso.LookupHost(ctx, domain) + if err != nil { + return nil, err + } + + const delay = 300 * time.Millisecond + var tactics []HTTPSDialerTactic + for idx, addr := range addrs { + tactics = append(tactics, &httpsDialerNullTactic{ + Address: addr, + Delay: time.Duration(idx) * delay, + Domain: domain, + }) + } + + return tactics, nil +} + +// Parallelism implements HTTPSDialerPolicy. +func (*HTTPSDialerNullPolicy) Parallelism() int { + return 16 +} + +// httpsDialerNullTactic is the default "null" tactic where we use the +// resolved IP addresses with the domain as the SNI value. +// +// We say that this is the "null" tactic because this is what you would get +// by default if you were not using any tactic. +type httpsDialerNullTactic struct { + // Address is the IP address we resolved. + Address string + + // Delay is the delay after which we start dialing. + Delay time.Duration + + // Domain is the related IP address. + Domain string +} + +// IPAddr implements HTTPSDialerTactic. +func (dt *httpsDialerNullTactic) IPAddr() string { + return dt.Address +} + +// InitialDelay implements HTTPSDialerTactic. +func (dt *httpsDialerNullTactic) InitialDelay() time.Duration { + return dt.Delay +} + +// NewTLSHandshaker implements HTTPSDialerTactic. +func (*httpsDialerNullTactic) NewTLSHandshaker(netx *netxlite.Netx, logger model.Logger) model.TLSHandshaker { + return netx.NewTLSHandshakerStdlib(logger) +} + +// OnStarting implements HTTPSDialerTactic. +func (*httpsDialerNullTactic) OnStarting() { + // nothing +} + +// OnSuccess implements HTTPSDialerTactic. +func (*httpsDialerNullTactic) OnSuccess() { + // nothing +} + +// OnTCPConnectError implements HTTPSDialerTactic. +func (*httpsDialerNullTactic) OnTCPConnectError(err error) { + // nothing +} + +// OnTLSHandshakeError implements HTTPSDialerTactic. +func (*httpsDialerNullTactic) OnTLSHandshakeError(err error) { + // nothing +} + +// OnTLSVerifyError implements HTTPSDialerTactic. +func (*httpsDialerNullTactic) OnTLSVerifyError(err error) { + // nothing +} + +// SNI implements HTTPSDialerTactic. +func (dt *httpsDialerNullTactic) SNI() string { + return dt.Domain +} + +// String implements fmt.Stringer. +func (dt *httpsDialerNullTactic) String() string { + return fmt.Sprintf("NullTactic{Address:\"%s\" Domain:\"%s\"}", dt.Address, dt.Domain) +} + +// VerifyHostname implements HTTPSDialerTactic. +func (dt *httpsDialerNullTactic) VerifyHostname() string { + return dt.Domain +} + +// HTTPSDialer is the [model.TLSDialer] used by the engine to dial HTTPS connections. +// +// The zero value of this struct is invalid; construct using [NewHTTPSDialer]. +// +// This dialer MAY use an happy-eyeballs-like policy where we may try several IP addresses, +// including IPv4 and IPv6, and dialing tactics in parallel. +type HTTPSDialer struct { + // idGenerator is the ID generator. + idGenerator *atomic.Int64 + + // logger is the logger to use. + logger model.Logger + + // policy defines the dialing policy to use. + policy HTTPSDialerPolicy + + // resolver is the DNS resolver to use. + resolver model.Resolver + + // rootCAs contains the root certificate pool we should use. + rootCAs *x509.CertPool + + // unet is the underlying network. + unet model.UnderlyingNetwork + + // wg is the wait group for knowing when all goroutines + // started in the background joined (for testing). + wg *sync.WaitGroup +} + +// NewHTTPSDialer constructs a new [*HTTPSDialer] instance. +// +// Arguments: +// +// - logger is the logger to use for logging; +// +// - policy defines the dialer policy; +// +// - resolver is the resolver to use; +// +// - unet is the underlying network to use. +// +// The returned [*HTTPSDialer] would use the underlying network's +// DefaultCertPool to create and cache the cert pool to use. +func NewHTTPSDialer( + logger model.Logger, + policy HTTPSDialerPolicy, + resolver model.Resolver, + unet model.UnderlyingNetwork, +) *HTTPSDialer { + return &HTTPSDialer{ + idGenerator: &atomic.Int64{}, + logger: &logx.PrefixLogger{ + Prefix: "HTTPSDialer: ", + Logger: logger, + }, + policy: policy, + resolver: resolver, + rootCAs: unet.DefaultCertPool(), + unet: unet, + wg: &sync.WaitGroup{}, + } +} + +var _ model.TLSDialer = &HTTPSDialer{} + +// WaitGroup returns the [*sync.WaitGroup] tracking the number of background goroutines. +func (hd *HTTPSDialer) WaitGroup() *sync.WaitGroup { + return hd.wg +} + +// CloseIdleConnections implements model.TLSDialer. +func (hd *HTTPSDialer) CloseIdleConnections() { + hd.resolver.CloseIdleConnections() +} + +// httpsDialerErrorOrConn contains either an error or a valid conn. +type httpsDialerErrorOrConn struct { + // Conn is the established TLS conn or nil. + Conn model.TLSConn + + // Err is the error or nil. + Err error +} + +// DialTLSContext implements model.TLSDialer. +func (hd *HTTPSDialer) DialTLSContext(ctx context.Context, network string, endpoint string) (net.Conn, error) { + hostname, port, err := net.SplitHostPort(endpoint) + if err != nil { + return nil, err + } + + // We need a cancellable context to interrupt the tactics emitter early when we + // immediately get a valid response and we don't need to use other tactics. + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + logger := &logx.PrefixLogger{ + Prefix: fmt.Sprintf("[#%d] ", hd.idGenerator.Add(1)), + Logger: hd.logger, + } + ol := logx.NewOperationLogger(logger, "LookupTactics: %s", hostname) + tactics, err := hd.policy.LookupTactics(ctx, hostname, hd.resolver) + if err != nil { + ol.Stop(err) + return nil, err + } + ol.Stop(tactics) + + emitter := hd.tacticsEmitter(ctx, tactics...) + collector := make(chan *httpsDialerErrorOrConn) + + parallelism := hd.policy.Parallelism() + runtimex.Assert(parallelism >= 1, "expected parallelism to be >= 1") + for idx := 0; idx < parallelism; idx++ { + hd.wg.Add(1) + go func() { + defer hd.wg.Done() + hd.worker(ctx, hostname, emitter, port, collector) + }() + } + + var ( + numDials = len(tactics) + errorv = []error{} + ) + for idx := 0; idx < numDials; idx++ { + select { + case <-ctx.Done(): + return nil, ctx.Err() + + case result := <-collector: + if result.Err != nil { + errorv = append(errorv, result.Err) + continue + } + return result.Conn, nil + } + } + + return nil, errors.Join(errorv...) +} + +// tacticsEmitter returns a channel closed once we have emitted all the tactics or the context is done. +func (hd *HTTPSDialer) tacticsEmitter(ctx context.Context, tactics ...HTTPSDialerTactic) <-chan HTTPSDialerTactic { + out := make(chan HTTPSDialerTactic) + + hd.wg.Add(1) + go func() { + defer hd.wg.Done() + defer close(out) + + for _, tactic := range tactics { + select { + case out <- tactic: + continue + + case <-ctx.Done(): + return + } + } + }() + + return out +} + +// worker attempts to establish a TLS connection using and emits a single +// [*httpsDialerErrorOrConn] for each tactic. +func (hd *HTTPSDialer) worker( + ctx context.Context, + hostname string, + reader <-chan HTTPSDialerTactic, + port string, + writer chan<- *httpsDialerErrorOrConn, +) { + for { + select { + case tactic, good := <-reader: + if !good { + // This happens when the emitter goroutine has closed the channel + return + } + + logger := &logx.PrefixLogger{ + Prefix: fmt.Sprintf("[#%d] ", hd.idGenerator.Add(1)), + Logger: hd.logger, + } + conn, err := hd.dialTLS(ctx, logger, tactic, port) + + select { + case <-ctx.Done(): + if conn != nil { + conn.Close() // we own the connection + } + return + + case writer <- &httpsDialerErrorOrConn{Conn: conn, Err: err}: + continue + } + + case <-ctx.Done(): + return + } + } +} + +// dialTLS performs the actual TLS dial. +func (hd *HTTPSDialer) dialTLS(ctx context.Context, + logger model.Logger, tactic HTTPSDialerTactic, port string) (model.TLSConn, error) { + // wait for the tactic to be ready to run + if err := httpsDialerTacticWaitReady(ctx, tactic); err != nil { + return nil, err + } + + // tell the tactic that we're starting + tactic.OnStarting() + + // create a network abstraction using the underlying network + netx := &netxlite.Netx{Underlying: hd.unet} + + // create dialer and establish TCP connection + endpoint := net.JoinHostPort(tactic.IPAddr(), port) + ol := logx.NewOperationLogger(logger, "TCPConnect %s", endpoint) + dialer := netx.NewDialerWithoutResolver(logger) + tcpConn, err := dialer.DialContext(ctx, "tcp", endpoint) + ol.Stop(err) + + // handle a dialing error + if err != nil { + tactic.OnTCPConnectError(err) + return nil, err + } + + // create TLS configuration + tlsConfig := &tls.Config{ + InsecureSkipVerify: true, // Note: we're going to verify at the end of the func + NextProtos: []string{"h2", "http/1.1"}, + RootCAs: hd.rootCAs, + ServerName: tactic.SNI(), + } + + // create handshaker and establish a TLS connection + ol = logx.NewOperationLogger( + logger, + "TLSHandshake with %s SNI=%s ALPN=%v", + endpoint, + tlsConfig.ServerName, + tlsConfig.NextProtos, + ) + thx := tactic.NewTLSHandshaker(netx, logger) + tlsConn, err := thx.Handshake(ctx, tcpConn, tlsConfig) + ol.Stop(err) + + // handle handshake error + if err != nil { + tactic.OnTLSHandshakeError(err) + tcpConn.Close() + return nil, err + } + + // verify the certificate chain + ol = logx.NewOperationLogger(logger, "TLSVerifyCertificateChain %s", tactic.VerifyHostname()) + err = httpsDialerVerifyCertificateChain(tactic.VerifyHostname(), tlsConn, hd.rootCAs) + ol.Stop(err) + + // handle verification error + if err != nil { + tactic.OnTLSVerifyError(err) + tlsConn.Close() + return nil, err + } + + // make sure the tactic know it worked + tactic.OnSuccess() + + return tlsConn, nil +} + +// httpsDialerWaitReady waits for the given delay to expire or the context to be canceled. If the +// delay is zero or negative, we immediately return nil. We also return nil when the delay expires. We +// return the context error if the context expires. +func httpsDialerTacticWaitReady(ctx context.Context, tactic HTTPSDialerTactic) error { + delay := tactic.InitialDelay() + if delay <= 0 { + return nil + } + + timer := time.NewTimer(delay) + defer timer.Stop() + + select { + case <-timer.C: + return nil + + case <-ctx.Done(): + return ctx.Err() + } +} + +// httpsDialerVerifyCertificateChain verifies the certificate chain with the given hostname. +func httpsDialerVerifyCertificateChain(hostname string, conn model.TLSConn, rootCAs *x509.CertPool) error { + // This code comes from the example in the Go source tree that shows + // how to override certificate verification and which is advertised + // as follows: + // + // VerifyConnection can be used to replace and customize connection + // verification. This example shows a VerifyConnection implementation that + // will be approximately equivalent to what crypto/tls does normally to + // verify the peer's certificate. + // + // See https://github.com/golang/go/blob/go1.21.0/src/crypto/tls/example_test.go#L186 + // + // As of go1.21.0, the code we're replacing has approximately the same + // implementation of the verification code we added below. + // + // See https://github.com/golang/go/blob/go1.21.0/src/crypto/tls/handshake_client.go#L962. + + state := conn.ConnectionState() + opts := x509.VerifyOptions{ + DNSName: hostname, // note: here we're using the real hostname + Intermediates: x509.NewCertPool(), + Roots: rootCAs, + } + for _, cert := range state.PeerCertificates[1:] { + opts.Intermediates.AddCert(cert) + } + + if _, err := state.PeerCertificates[0].Verify(opts); err != nil { + return netxlite.NewErrWrapper(netxlite.ClassifyTLSHandshakeError, netxlite.TopLevelOperation, err) + } + return nil +} diff --git a/internal/enginenetx/httpsdialer_test.go b/internal/enginenetx/httpsdialer_test.go new file mode 100644 index 0000000000..e01b039844 --- /dev/null +++ b/internal/enginenetx/httpsdialer_test.go @@ -0,0 +1,352 @@ +package enginenetx_test + +import ( + "context" + "testing" + + "github.com/apex/log" + "github.com/google/go-cmp/cmp" + "github.com/ooni/netem" + "github.com/ooni/probe-cli/v3/internal/enginenetx" + "github.com/ooni/probe-cli/v3/internal/model" + "github.com/ooni/probe-cli/v3/internal/netemx" + "github.com/ooni/probe-cli/v3/internal/netxlite" +) + +// Flags controlling when [httpsDialerPolicyCancelingContext] cancels the context +const ( + httpsDialerPolicyCancelingContextOnStarting = 1 << iota + httpsDialerPolicyCancelingContextOnSuccess +) + +// httpsDialerPolicyCancelingContext is an [enginenetsx.HTTPSDialerPolicy] with a cancel +// function that causes the context to be canceled once we start dialing. +// +// This struct helps with testing [enginenetx.HTTPSDialer] is WAI when the context +// has been canceled and we correctly shutdown all goroutines. +type httpsDialerPolicyCancelingContext struct { + cancelOnStarting context.CancelFunc + cancelOnSuccess context.CancelFunc + flags int + policy enginenetx.HTTPSDialerPolicy +} + +var _ enginenetx.HTTPSDialerPolicy = &httpsDialerPolicyCancelingContext{} + +// LookupTactics implements enginenetx.HTTPSDialerPolicy. +func (p *httpsDialerPolicyCancelingContext) LookupTactics(ctx context.Context, domain string, reso model.Resolver) ([]enginenetx.HTTPSDialerTactic, error) { + tactics, err := p.policy.LookupTactics(ctx, domain, reso) + if err != nil { + return nil, err + } + var out []enginenetx.HTTPSDialerTactic + for _, tactic := range tactics { + out = append(out, &httpsDialerTacticCancelingContext{ + HTTPSDialerTactic: tactic, + cancelOnStarting: p.cancelOnStarting, + cancelOnSuccess: p.cancelOnSuccess, + }) + } + return out, nil +} + +// Parallelism implements enginenetx.HTTPSDialerPolicy. +func (p *httpsDialerPolicyCancelingContext) Parallelism() int { + return p.policy.Parallelism() +} + +// httpsDialerTacticCancelingContext is the tactic returned by [httpsDialerPolicyCancelingContext]. +type httpsDialerTacticCancelingContext struct { + enginenetx.HTTPSDialerTactic + cancelOnStarting context.CancelFunc + cancelOnSuccess context.CancelFunc +} + +// OnStarting implements enginenetx.HTTPSDialerTactic. +func (t *httpsDialerTacticCancelingContext) OnStarting() { + if t.cancelOnStarting != nil { + t.cancelOnStarting() + } +} + +// OnSuccess implements enginenetx.HTTPSDialerTactic. +func (t *httpsDialerTacticCancelingContext) OnSuccess() { + if t.cancelOnSuccess != nil { + t.cancelOnSuccess() + } +} + +func TestHTTPSDialerWAI(t *testing.T) { + // testcase is a test case implemented by this function + type testcase struct { + // name is the name of the test case + name string + + // short indicates whether this is a short test + short bool + + // policy is the dialer policy + policy enginenetx.HTTPSDialerPolicy + + // endpoint is the endpoint to connect to consisting of a domain + // name or IP address followed by a TCP port + endpoint string + + // scenario is the netemx testing scenario to create + scenario []*netemx.ScenarioDomainAddresses + + // configureDPI configures DPI rules (just add an empty + // function if you don't need any) + configureDPI func(dpi *netem.DPIEngine) + + // expectErr is the error string we expect to see + expectErr string + } + + allTestCases := []testcase{{ + name: "net.SplitHostPort failure", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com", // note: here the port is missing + scenario: netemx.InternetScenario, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing + }, + expectErr: "address www.example.com: missing port in address", + }, { + name: "hd.policy.LookupTactics failure", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.nonexistent:443", // note: the domain does not exist + scenario: netemx.InternetScenario, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing + }, + expectErr: "dns_nxdomain_error", + }, { + name: "successful dial with multiple addresses", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + "93.184.216.36", + "93.184.216.37", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing + }, + expectErr: "", + }, { + name: "with TCP connect errors", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // we force closing the connection for all the known server endpoints + dpi.AddRule(&netem.DPICloseConnectionForServerEndpoint{ + Logger: log.Log, + ServerIPAddress: "93.184.216.34", + ServerPort: 443, + }) + dpi.AddRule(&netem.DPICloseConnectionForServerEndpoint{ + Logger: log.Log, + ServerIPAddress: "93.184.216.35", + ServerPort: 443, + }) + }, + expectErr: "connection_refused\nconnection_refused", + }, { + name: "with TLS handshake errors", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // we force resetting the connection for www.example.com + dpi.AddRule(&netem.DPIResetTrafficForTLSSNI{ + Logger: log.Log, + SNI: "www.example.com", + }) + }, + expectErr: "connection_reset\nconnection_reset", + }, { + name: "with TLS verification errors", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "wrong.host.badssl.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "wrong.host.badssl.com", + "untrusted-root.badssl.com", + "expired.badssl.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleBadSSL, + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing + }, + expectErr: "ssl_invalid_hostname\nssl_invalid_hostname", + }, { + name: "with context being canceled in OnStarting", + short: true, + policy: &httpsDialerPolicyCancelingContext{ + cancelOnStarting: nil, + cancelOnSuccess: nil, + flags: httpsDialerPolicyCancelingContextOnStarting, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + }, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing + }, + expectErr: "context canceled", + }, { + name: "with context being canceled in OnSuccess for the first success", + short: true, + policy: &httpsDialerPolicyCancelingContext{ + cancelOnStarting: nil, + cancelOnSuccess: nil, + flags: httpsDialerPolicyCancelingContextOnSuccess, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + }, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing + }, + expectErr: "context canceled", + }} + + for _, tc := range allTestCases { + t.Run(tc.name, func(t *testing.T) { + // make sure we honor `go test -short` + if !tc.short && testing.Short() { + t.Skip("skip test in short mode") + } + + // create the QA environment + env := netemx.MustNewScenario(tc.scenario) + defer env.Close() + + // possibly add specific DPI rules + tc.configureDPI(env.DPIEngine()) + + // create the proper underlying network + unet := &netxlite.NetemUnderlyingNetworkAdapter{UNet: env.ClientStack} + + // create the network proper + netx := &netxlite.Netx{Underlying: unet} + + // create the getaddrinfo resolver + resolver := netx.NewStdlibResolver(log.Log) + + // create the TLS dialer + dialer := enginenetx.NewHTTPSDialer( + log.Log, + tc.policy, + resolver, + unet, + ) + defer dialer.CloseIdleConnections() + + // configure context and possibly add timeout + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // Possibly tell the httpsDialerPolicyCancelingContext about the cancel func + // depending on which flags have been configured. + if p, ok := tc.policy.(*httpsDialerPolicyCancelingContext); ok { + if (p.flags & httpsDialerPolicyCancelingContextOnStarting) != 0 { + p.cancelOnStarting = cancel + } + if (p.flags & httpsDialerPolicyCancelingContextOnSuccess) != 0 { + p.cancelOnSuccess = cancel + } + } + + // dial the TLS connection + tlsConn, err := dialer.DialTLSContext(ctx, "tcp", tc.endpoint) + + t.Logf("%+v %+v", tlsConn, err) + + // make sure the error is the one we expected + switch { + case err != nil && tc.expectErr == "": + t.Fatal("expected", tc.expectErr, "got", err) + + case err == nil && tc.expectErr != "": + t.Fatal("expected", tc.expectErr, "got", err) + + case err != nil && tc.expectErr != "": + if diff := cmp.Diff(tc.expectErr, err.Error()); diff != "" { + t.Fatal(diff) + } + return + + case err == nil && tc.expectErr == "": + // all good + } + + // make sure we close the conn + defer tlsConn.Close() + + // wait for background connections to join + dialer.WaitGroup().Wait() + }) + } +} From 777da7c3c099c736b3b226e8c553607af99dca7b Mon Sep 17 00:00:00 2001 From: Simone Basso Date: Wed, 20 Sep 2023 11:27:03 +0200 Subject: [PATCH 2/7] x --- internal/enginenetx/httpsdialer.go | 27 +++++++++++++++++-- .../enginenetx/httpsdialer_internal_test.go | 25 +++++++++++++++++ 2 files changed, 50 insertions(+), 2 deletions(-) create mode 100644 internal/enginenetx/httpsdialer_internal_test.go diff --git a/internal/enginenetx/httpsdialer.go b/internal/enginenetx/httpsdialer.go index 0ae9438f68..a4f82e73e0 100644 --- a/internal/enginenetx/httpsdialer.go +++ b/internal/enginenetx/httpsdialer.go @@ -21,6 +21,9 @@ import ( type HTTPSDialerPolicy interface { // LookupTactics performs a DNS lookup for the given domain using the given resolver and // returns either a list of tactics for dialing or an error. + // + // This functoion MUST NOT return an empty list and a nil error. If this happens the + // code inside [HTTPSDialer] will panic. LookupTactics(ctx context.Context, domain string, reso model.Resolver) ([]HTTPSDialerTactic, error) // Parallelism returns the number of goroutines to create when TLS dialing. The @@ -88,7 +91,7 @@ func (*HTTPSDialerNullPolicy) LookupTactics( for idx, addr := range addrs { tactics = append(tactics, &httpsDialerNullTactic{ Address: addr, - Delay: time.Duration(idx) * delay, + Delay: time.Duration(idx) * delay, // zero for the first dial Domain: domain, }) } @@ -238,7 +241,8 @@ func NewHTTPSDialer( var _ model.TLSDialer = &HTTPSDialer{} -// WaitGroup returns the [*sync.WaitGroup] tracking the number of background goroutines. +// WaitGroup returns the [*sync.WaitGroup] tracking the number of background goroutines, +// which is definitely useful in testing to make sure we join all the goroutines. func (hd *HTTPSDialer) WaitGroup() *sync.WaitGroup { return hd.wg } @@ -280,6 +284,7 @@ func (hd *HTTPSDialer) DialTLSContext(ctx context.Context, network string, endpo return nil, err } ol.Stop(tactics) + runtimex.Assert(len(tactics) >= 1, "expected at least one tactic here") emitter := hd.tacticsEmitter(ctx, tactics...) collector := make(chan *httpsDialerErrorOrConn) @@ -308,6 +313,10 @@ func (hd *HTTPSDialer) DialTLSContext(ctx context.Context, network string, endpo errorv = append(errorv, result.Err) continue } + + // Returning early cancels the context and this cancellation + // causes other background goroutines to interrupt their long + // running network operations or unblocks them while sending return result.Conn, nil } } @@ -347,6 +356,9 @@ func (hd *HTTPSDialer) worker( port string, writer chan<- *httpsDialerErrorOrConn, ) { + // Note: no need to be concerned with the wait group here because + // we're managing it inside DialTLSContext so Add and Done live together + for { select { case tactic, good := <-reader: @@ -471,6 +483,9 @@ func httpsDialerTacticWaitReady(ctx context.Context, tactic HTTPSDialerTactic) e } } +// errNoPeerCertificate is an internal error returned when we don't have any peer certificate. +var errNoPeerCertificate = errors.New("no peer certificate") + // httpsDialerVerifyCertificateChain verifies the certificate chain with the given hostname. func httpsDialerVerifyCertificateChain(hostname string, conn model.TLSConn, rootCAs *x509.CertPool) error { // This code comes from the example in the Go source tree that shows @@ -499,6 +514,14 @@ func httpsDialerVerifyCertificateChain(hostname string, conn model.TLSConn, root opts.Intermediates.AddCert(cert) } + // The following check is rather paranoid and it's not part of the Go codebase + // from which we copied it, but I think it's important to be defensive. + // + // Because of that, I don't want to just drop an assertion here. + if len(state.PeerCertificates) < 1 { + return errNoPeerCertificate + } + if _, err := state.PeerCertificates[0].Verify(opts); err != nil { return netxlite.NewErrWrapper(netxlite.ClassifyTLSHandshakeError, netxlite.TopLevelOperation, err) } diff --git a/internal/enginenetx/httpsdialer_internal_test.go b/internal/enginenetx/httpsdialer_internal_test.go new file mode 100644 index 0000000000..4884ca3d34 --- /dev/null +++ b/internal/enginenetx/httpsdialer_internal_test.go @@ -0,0 +1,25 @@ +package enginenetx + +import ( + "crypto/tls" + "errors" + "testing" + + "github.com/ooni/probe-cli/v3/internal/mocks" + "github.com/ooni/probe-cli/v3/internal/netxlite" +) + +func TestHTTPSDialerVerifyCertificateChain(t *testing.T) { + t.Run("without any peer certificate", func(t *testing.T) { + tlsConn := &mocks.TLSConn{ + MockConnectionState: func() tls.ConnectionState { + return tls.ConnectionState{} // empty! + }, + } + certPool := netxlite.NewMozillaCertPool() + err := httpsDialerVerifyCertificateChain("www.example.com", tlsConn, certPool) + if !errors.Is(err, errNoPeerCertificate) { + t.Fatal("unexpected error", err) + } + }) +} From e24083702997929a17d72846542ce11c18349623 Mon Sep 17 00:00:00 2001 From: Simone Basso Date: Wed, 20 Sep 2023 11:28:21 +0200 Subject: [PATCH 3/7] x --- internal/enginenetx/httpsdialer.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/internal/enginenetx/httpsdialer.go b/internal/enginenetx/httpsdialer.go index a4f82e73e0..837230a9be 100644 --- a/internal/enginenetx/httpsdialer.go +++ b/internal/enginenetx/httpsdialer.go @@ -510,9 +510,6 @@ func httpsDialerVerifyCertificateChain(hostname string, conn model.TLSConn, root Intermediates: x509.NewCertPool(), Roots: rootCAs, } - for _, cert := range state.PeerCertificates[1:] { - opts.Intermediates.AddCert(cert) - } // The following check is rather paranoid and it's not part of the Go codebase // from which we copied it, but I think it's important to be defensive. @@ -522,6 +519,10 @@ func httpsDialerVerifyCertificateChain(hostname string, conn model.TLSConn, root return errNoPeerCertificate } + for _, cert := range state.PeerCertificates[1:] { + opts.Intermediates.AddCert(cert) + } + if _, err := state.PeerCertificates[0].Verify(opts); err != nil { return netxlite.NewErrWrapper(netxlite.ClassifyTLSHandshakeError, netxlite.TopLevelOperation, err) } From ae275eb768fd685a1c0560744218ab41ba8832c4 Mon Sep 17 00:00:00 2001 From: Simone Basso Date: Wed, 20 Sep 2023 11:34:56 +0200 Subject: [PATCH 4/7] x --- internal/enginenetx/httpsdialer_test.go | 44 ++++++++++--------------- 1 file changed, 18 insertions(+), 26 deletions(-) diff --git a/internal/enginenetx/httpsdialer_test.go b/internal/enginenetx/httpsdialer_test.go index e01b039844..dce16e09f1 100644 --- a/internal/enginenetx/httpsdialer_test.go +++ b/internal/enginenetx/httpsdialer_test.go @@ -25,10 +25,9 @@ const ( // This struct helps with testing [enginenetx.HTTPSDialer] is WAI when the context // has been canceled and we correctly shutdown all goroutines. type httpsDialerPolicyCancelingContext struct { - cancelOnStarting context.CancelFunc - cancelOnSuccess context.CancelFunc - flags int - policy enginenetx.HTTPSDialerPolicy + cancel context.CancelFunc + flags int + policy enginenetx.HTTPSDialerPolicy } var _ enginenetx.HTTPSDialerPolicy = &httpsDialerPolicyCancelingContext{} @@ -43,8 +42,8 @@ func (p *httpsDialerPolicyCancelingContext) LookupTactics(ctx context.Context, d for _, tactic := range tactics { out = append(out, &httpsDialerTacticCancelingContext{ HTTPSDialerTactic: tactic, - cancelOnStarting: p.cancelOnStarting, - cancelOnSuccess: p.cancelOnSuccess, + cancel: p.cancel, + flags: p.flags, }) } return out, nil @@ -58,21 +57,21 @@ func (p *httpsDialerPolicyCancelingContext) Parallelism() int { // httpsDialerTacticCancelingContext is the tactic returned by [httpsDialerPolicyCancelingContext]. type httpsDialerTacticCancelingContext struct { enginenetx.HTTPSDialerTactic - cancelOnStarting context.CancelFunc - cancelOnSuccess context.CancelFunc + cancel context.CancelFunc + flags int } // OnStarting implements enginenetx.HTTPSDialerTactic. func (t *httpsDialerTacticCancelingContext) OnStarting() { - if t.cancelOnStarting != nil { - t.cancelOnStarting() + if (t.flags & httpsDialerPolicyCancelingContextOnStarting) != 0 { + t.cancel() } } // OnSuccess implements enginenetx.HTTPSDialerTactic. func (t *httpsDialerTacticCancelingContext) OnSuccess() { - if t.cancelOnSuccess != nil { - t.cancelOnSuccess() + if (p.flags & httpsDialerPolicyCancelingContextOnSuccess) != 0 { + t.cancel() } } @@ -224,10 +223,9 @@ func TestHTTPSDialerWAI(t *testing.T) { name: "with context being canceled in OnStarting", short: true, policy: &httpsDialerPolicyCancelingContext{ - cancelOnStarting: nil, - cancelOnSuccess: nil, - flags: httpsDialerPolicyCancelingContextOnStarting, - policy: &enginenetx.HTTPSDialerNullPolicy{}, + cancel: nil, + flags: httpsDialerPolicyCancelingContextOnStarting, + policy: &enginenetx.HTTPSDialerNullPolicy{}, }, endpoint: "www.example.com:443", scenario: []*netemx.ScenarioDomainAddresses{{ @@ -249,10 +247,9 @@ func TestHTTPSDialerWAI(t *testing.T) { name: "with context being canceled in OnSuccess for the first success", short: true, policy: &httpsDialerPolicyCancelingContext{ - cancelOnStarting: nil, - cancelOnSuccess: nil, - flags: httpsDialerPolicyCancelingContextOnSuccess, - policy: &enginenetx.HTTPSDialerNullPolicy{}, + cancel: nil, + flags: httpsDialerPolicyCancelingContextOnSuccess, + policy: &enginenetx.HTTPSDialerNullPolicy{}, }, endpoint: "www.example.com:443", scenario: []*netemx.ScenarioDomainAddresses{{ @@ -311,12 +308,7 @@ func TestHTTPSDialerWAI(t *testing.T) { // Possibly tell the httpsDialerPolicyCancelingContext about the cancel func // depending on which flags have been configured. if p, ok := tc.policy.(*httpsDialerPolicyCancelingContext); ok { - if (p.flags & httpsDialerPolicyCancelingContextOnStarting) != 0 { - p.cancelOnStarting = cancel - } - if (p.flags & httpsDialerPolicyCancelingContextOnSuccess) != 0 { - p.cancelOnSuccess = cancel - } + p.cancel = cancel } // dial the TLS connection From 7da263d6127556a1ec48b05843cf302c36496bc2 Mon Sep 17 00:00:00 2001 From: Simone Basso Date: Wed, 20 Sep 2023 11:35:13 +0200 Subject: [PATCH 5/7] x --- internal/enginenetx/httpsdialer_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/enginenetx/httpsdialer_test.go b/internal/enginenetx/httpsdialer_test.go index dce16e09f1..09c8da3a24 100644 --- a/internal/enginenetx/httpsdialer_test.go +++ b/internal/enginenetx/httpsdialer_test.go @@ -70,7 +70,7 @@ func (t *httpsDialerTacticCancelingContext) OnStarting() { // OnSuccess implements enginenetx.HTTPSDialerTactic. func (t *httpsDialerTacticCancelingContext) OnSuccess() { - if (p.flags & httpsDialerPolicyCancelingContextOnSuccess) != 0 { + if (t.flags & httpsDialerPolicyCancelingContextOnSuccess) != 0 { t.cancel() } } From 93288681fa18c3da302401e86bc434ce360b4e7f Mon Sep 17 00:00:00 2001 From: Simone Basso Date: Wed, 20 Sep 2023 11:47:59 +0200 Subject: [PATCH 6/7] x --- .../enginenetx/httpsdialer_internal_test.go | 45 +++++++++++++++++++ internal/enginenetx/httpsdialer_test.go | 10 +++-- 2 files changed, 52 insertions(+), 3 deletions(-) diff --git a/internal/enginenetx/httpsdialer_internal_test.go b/internal/enginenetx/httpsdialer_internal_test.go index 4884ca3d34..37ff85980d 100644 --- a/internal/enginenetx/httpsdialer_internal_test.go +++ b/internal/enginenetx/httpsdialer_internal_test.go @@ -1,14 +1,59 @@ package enginenetx import ( + "context" "crypto/tls" "errors" + "sync" + "sync/atomic" "testing" "github.com/ooni/probe-cli/v3/internal/mocks" + "github.com/ooni/probe-cli/v3/internal/model" "github.com/ooni/probe-cli/v3/internal/netxlite" ) +func TestHTTPSDialerTacticsEmitter(t *testing.T) { + t.Run("we correctly handle the case of a canceled context", func(t *testing.T) { + hd := &HTTPSDialer{ + idGenerator: &atomic.Int64{}, + logger: model.DiscardLogger, + policy: &HTTPSDialerNullPolicy{}, + resolver: netxlite.NewStdlibResolver(model.DiscardLogger), + rootCAs: netxlite.NewMozillaCertPool(), + unet: &netxlite.DefaultTProxy{}, + wg: &sync.WaitGroup{}, + } + + tactics := []HTTPSDialerTactic{ + &httpsDialerNullTactic{ + Address: "10.0.0.1", + Delay: 0, + Domain: "www.example.com", + }, + &httpsDialerNullTactic{ + Address: "10.0.0.2", + Delay: 0, + Domain: "www.example.com", + }, + } + + ctx, cancel := context.WithCancel(context.Background()) + cancel() // we want the tested function to run with a canceled context + + out := hd.tacticsEmitter(ctx, tactics...) + + var count int + for range out { + count++ + } + + if count != 0 { + t.Fatal("nothing should have been emitted here") + } + }) +} + func TestHTTPSDialerVerifyCertificateChain(t *testing.T) { t.Run("without any peer certificate", func(t *testing.T) { tlsConn := &mocks.TLSConn{ diff --git a/internal/enginenetx/httpsdialer_test.go b/internal/enginenetx/httpsdialer_test.go index 09c8da3a24..0ccdf48110 100644 --- a/internal/enginenetx/httpsdialer_test.go +++ b/internal/enginenetx/httpsdialer_test.go @@ -199,6 +199,9 @@ func TestHTTPSDialerWAI(t *testing.T) { }, expectErr: "connection_reset\nconnection_reset", }, { + // Note: this is where we test that TLS verification is WAI. The netemx scenario role + // constructs the equivalent of real world's badssl.com and we're checking whether + // we would accept a certificate valid for another hostname. The answer should be "NO!". name: "with TLS verification errors", short: true, policy: &enginenetx.HTTPSDialerNullPolicy{}, @@ -301,7 +304,7 @@ func TestHTTPSDialerWAI(t *testing.T) { ) defer dialer.CloseIdleConnections() - // configure context and possibly add timeout + // configure cancellable context--some tests are going to use cancel ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -328,14 +331,15 @@ func TestHTTPSDialerWAI(t *testing.T) { if diff := cmp.Diff(tc.expectErr, err.Error()); diff != "" { t.Fatal(diff) } - return case err == nil && tc.expectErr == "": // all good } // make sure we close the conn - defer tlsConn.Close() + if tlsConn != nil { + defer tlsConn.Close() + } // wait for background connections to join dialer.WaitGroup().Wait() From 10aa5ce4189db0f2dcae542e2b4854c5e0ca3eef Mon Sep 17 00:00:00 2001 From: Simone Basso Date: Wed, 20 Sep 2023 11:54:59 +0200 Subject: [PATCH 7/7] x --- internal/enginenetx/httpsdialer_test.go | 374 ++++++++++++++---------- 1 file changed, 225 insertions(+), 149 deletions(-) diff --git a/internal/enginenetx/httpsdialer_test.go b/internal/enginenetx/httpsdialer_test.go index 0ccdf48110..b3a6eb2c92 100644 --- a/internal/enginenetx/httpsdialer_test.go +++ b/internal/enginenetx/httpsdialer_test.go @@ -102,175 +102,251 @@ func TestHTTPSDialerWAI(t *testing.T) { expectErr string } - allTestCases := []testcase{{ - name: "net.SplitHostPort failure", - short: true, - policy: &enginenetx.HTTPSDialerNullPolicy{}, - endpoint: "www.example.com", // note: here the port is missing - scenario: netemx.InternetScenario, - configureDPI: func(dpi *netem.DPIEngine) { - // nothing - }, - expectErr: "address www.example.com: missing port in address", - }, { - name: "hd.policy.LookupTactics failure", - short: true, - policy: &enginenetx.HTTPSDialerNullPolicy{}, - endpoint: "www.example.nonexistent:443", // note: the domain does not exist - scenario: netemx.InternetScenario, - configureDPI: func(dpi *netem.DPIEngine) { - // nothing - }, - expectErr: "dns_nxdomain_error", - }, { - name: "successful dial with multiple addresses", - short: true, - policy: &enginenetx.HTTPSDialerNullPolicy{}, - endpoint: "www.example.com:443", - scenario: []*netemx.ScenarioDomainAddresses{{ - Domains: []string{ - "www.example.com", - }, - Addresses: []string{ - "93.184.216.34", - "93.184.216.35", - "93.184.216.36", - "93.184.216.37", + allTestCases := []testcase{ + + // This test case ensures that we handle the corner case of a missing port + { + name: "net.SplitHostPort failure", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com", // note: here the port is missing + scenario: netemx.InternetScenario, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing }, - Role: netemx.ScenarioRoleWebServer, - WebServerFactory: netemx.ExampleWebPageHandlerFactory(), - }}, - configureDPI: func(dpi *netem.DPIEngine) { - // nothing + expectErr: "address www.example.com: missing port in address", }, - expectErr: "", - }, { - name: "with TCP connect errors", - short: true, - policy: &enginenetx.HTTPSDialerNullPolicy{}, - endpoint: "www.example.com:443", - scenario: []*netemx.ScenarioDomainAddresses{{ - Domains: []string{ - "www.example.com", + + // This test case ensures that we handle the case of a nonexistent domain + { + name: "hd.policy.LookupTactics failure", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.nonexistent:443", // note: the domain does not exist + scenario: netemx.InternetScenario, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing }, - Addresses: []string{ - "93.184.216.34", - "93.184.216.35", + expectErr: "dns_nxdomain_error", + }, + + // This test case is the common case: all is good with multiple addresses to dial (I am + // not testing the case of a single address because it's a subcase of this one) + { + name: "successful dial with multiple addresses", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + "93.184.216.36", + "93.184.216.37", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing }, - Role: netemx.ScenarioRoleWebServer, - WebServerFactory: netemx.ExampleWebPageHandlerFactory(), - }}, - configureDPI: func(dpi *netem.DPIEngine) { - // we force closing the connection for all the known server endpoints - dpi.AddRule(&netem.DPICloseConnectionForServerEndpoint{ - Logger: log.Log, - ServerIPAddress: "93.184.216.34", - ServerPort: 443, - }) - dpi.AddRule(&netem.DPICloseConnectionForServerEndpoint{ - Logger: log.Log, - ServerIPAddress: "93.184.216.35", - ServerPort: 443, - }) + expectErr: "", }, - expectErr: "connection_refused\nconnection_refused", - }, { - name: "with TLS handshake errors", - short: true, - policy: &enginenetx.HTTPSDialerNullPolicy{}, - endpoint: "www.example.com:443", - scenario: []*netemx.ScenarioDomainAddresses{{ - Domains: []string{ - "www.example.com", + + // Here we make sure that we're doing OK if the addresses are TCP-blocked + { + name: "with TCP connect errors", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // we force closing the connection for all the known server endpoints + dpi.AddRule(&netem.DPICloseConnectionForServerEndpoint{ + Logger: log.Log, + ServerIPAddress: "93.184.216.34", + ServerPort: 443, + }) + dpi.AddRule(&netem.DPICloseConnectionForServerEndpoint{ + Logger: log.Log, + ServerIPAddress: "93.184.216.35", + ServerPort: 443, + }) }, - Addresses: []string{ - "93.184.216.34", - "93.184.216.35", + expectErr: "connection_refused\nconnection_refused", + }, + + // Here we're making sure it's all WAI when there is TLS interference + { + name: "with TLS handshake errors", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // we force resetting the connection for www.example.com + dpi.AddRule(&netem.DPIResetTrafficForTLSSNI{ + Logger: log.Log, + SNI: "www.example.com", + }) }, - Role: netemx.ScenarioRoleWebServer, - WebServerFactory: netemx.ExampleWebPageHandlerFactory(), - }}, - configureDPI: func(dpi *netem.DPIEngine) { - // we force resetting the connection for www.example.com - dpi.AddRule(&netem.DPIResetTrafficForTLSSNI{ - Logger: log.Log, - SNI: "www.example.com", - }) + expectErr: "connection_reset\nconnection_reset", }, - expectErr: "connection_reset\nconnection_reset", - }, { + // Note: this is where we test that TLS verification is WAI. The netemx scenario role // constructs the equivalent of real world's badssl.com and we're checking whether // we would accept a certificate valid for another hostname. The answer should be "NO!". - name: "with TLS verification errors", - short: true, - policy: &enginenetx.HTTPSDialerNullPolicy{}, - endpoint: "wrong.host.badssl.com:443", - scenario: []*netemx.ScenarioDomainAddresses{{ - Domains: []string{ - "wrong.host.badssl.com", - "untrusted-root.badssl.com", - "expired.badssl.com", + { + name: "with a TLS certificate valid for ANOTHER domain", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "wrong.host.badssl.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "wrong.host.badssl.com", + "untrusted-root.badssl.com", + "expired.badssl.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleBadSSL, + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing }, - Addresses: []string{ - "93.184.216.34", - "93.184.216.35", + expectErr: "ssl_invalid_hostname\nssl_invalid_hostname", + }, + + // Note: this is another TLS related test case where we make sure that + // we can handle an untrusted root/self signed certificate + { + name: "with TLS certificate signed by an unknown authority", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "untrusted-root.badssl.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "wrong.host.badssl.com", + "untrusted-root.badssl.com", + "expired.badssl.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleBadSSL, + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing }, - Role: netemx.ScenarioRoleBadSSL, - }}, - configureDPI: func(dpi *netem.DPIEngine) { - // nothing + expectErr: "ssl_unknown_authority\nssl_unknown_authority", }, - expectErr: "ssl_invalid_hostname\nssl_invalid_hostname", - }, { - name: "with context being canceled in OnStarting", - short: true, - policy: &httpsDialerPolicyCancelingContext{ - cancel: nil, - flags: httpsDialerPolicyCancelingContextOnStarting, - policy: &enginenetx.HTTPSDialerNullPolicy{}, + + // Note: this is another TLS related test case where we make sure that + // we can handle a certificate that has now expired. + { + name: "with expired TLS certificate", + short: true, + policy: &enginenetx.HTTPSDialerNullPolicy{}, + endpoint: "expired.badssl.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "wrong.host.badssl.com", + "untrusted-root.badssl.com", + "expired.badssl.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleBadSSL, + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing + }, + expectErr: "ssl_invalid_certificate\nssl_invalid_certificate", }, - endpoint: "www.example.com:443", - scenario: []*netemx.ScenarioDomainAddresses{{ - Domains: []string{ - "www.example.com", + + // This is a corner case: what if the context is canceled after the DNS lookup + // but before we start dialing? Are we closing all goroutines and returning correctly? + { + name: "with context being canceled in OnStarting", + short: true, + policy: &httpsDialerPolicyCancelingContext{ + cancel: nil, + flags: httpsDialerPolicyCancelingContextOnStarting, + policy: &enginenetx.HTTPSDialerNullPolicy{}, }, - Addresses: []string{ - "93.184.216.34", - "93.184.216.35", + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing }, - Role: netemx.ScenarioRoleWebServer, - WebServerFactory: netemx.ExampleWebPageHandlerFactory(), - }}, - configureDPI: func(dpi *netem.DPIEngine) { - // nothing - }, - expectErr: "context canceled", - }, { - name: "with context being canceled in OnSuccess for the first success", - short: true, - policy: &httpsDialerPolicyCancelingContext{ - cancel: nil, - flags: httpsDialerPolicyCancelingContextOnSuccess, - policy: &enginenetx.HTTPSDialerNullPolicy{}, + expectErr: "context canceled", }, - endpoint: "www.example.com:443", - scenario: []*netemx.ScenarioDomainAddresses{{ - Domains: []string{ - "www.example.com", + + // This is another corner case: what happens if the context is canceled after we + // have a good connection but before we're able to report it to the caller? + { + name: "with context being canceled in OnSuccess for the first success", + short: true, + policy: &httpsDialerPolicyCancelingContext{ + cancel: nil, + flags: httpsDialerPolicyCancelingContextOnSuccess, + policy: &enginenetx.HTTPSDialerNullPolicy{}, }, - Addresses: []string{ - "93.184.216.34", - "93.184.216.35", + endpoint: "www.example.com:443", + scenario: []*netemx.ScenarioDomainAddresses{{ + Domains: []string{ + "www.example.com", + }, + Addresses: []string{ + "93.184.216.34", + "93.184.216.35", + }, + Role: netemx.ScenarioRoleWebServer, + WebServerFactory: netemx.ExampleWebPageHandlerFactory(), + }}, + configureDPI: func(dpi *netem.DPIEngine) { + // nothing }, - Role: netemx.ScenarioRoleWebServer, - WebServerFactory: netemx.ExampleWebPageHandlerFactory(), - }}, - configureDPI: func(dpi *netem.DPIEngine) { - // nothing - }, - expectErr: "context canceled", - }} + expectErr: "context canceled", + }} for _, tc := range allTestCases { t.Run(tc.name, func(t *testing.T) {