VA: Remove legacy HTTP-01 validation code. (#4102)

We're only using the simplified HTTP-01 code from `va/http.go` now 🎉 The old unit tests that still seem relevant are left in place in `va/va_test.go` instead of being moved to `va/http_test.go` to signal that they're a bit crufty and could probably use a separate cleanup. For now I'm hesitant to remove test coverage so I updated them in-place without moving them to a new home.

Resolves https://github.com/letsencrypt/boulder/issues/4089
This commit is contained in:
Daniel McCarney 2019-03-08 14:57:39 -05:00 committed by Roland Bracewell Shoemaker
parent cc4ce59d7d
commit 9f5c1b9e25
6 changed files with 53 additions and 466 deletions

View File

@ -4,9 +4,9 @@ package features
import "strconv"
const _FeatureFlag_name = "unusedPerformValidationRPCACME13KeyRolloverAllowRenewalFirstRLTLSSNIRevalidationCAAValidationMethodsCAAAccountURIProbeCTLogsSimplifiedVAHTTPHeadNonceStatusOKNewAuthorizationSchemaRevokeAtRASetIssuedNamesRenewalBitEarlyOrderRateLimit"
const _FeatureFlag_name = "unusedPerformValidationRPCACME13KeyRolloverSimplifiedVAHTTPAllowRenewalFirstRLTLSSNIRevalidationCAAValidationMethodsCAAAccountURIProbeCTLogsHeadNonceStatusOKNewAuthorizationSchemaRevokeAtRASetIssuedNamesRenewalBitEarlyOrderRateLimit"
var _FeatureFlag_index = [...]uint8{0, 6, 26, 43, 62, 80, 100, 113, 124, 140, 157, 179, 189, 213, 232}
var _FeatureFlag_index = [...]uint8{0, 6, 26, 43, 59, 78, 96, 116, 129, 140, 157, 179, 189, 213, 232}
func (i FeatureFlag) String() string {
if i < 0 || i >= FeatureFlag(len(_FeatureFlag_index)-1) {

View File

@ -14,6 +14,7 @@ const (
// Deprecated features, these can be removed once stripped from production configs
PerformValidationRPC
ACME13KeyRollover
SimplifiedVAHTTP
// Currently in-use features
AllowRenewalFirstRL
@ -25,9 +26,6 @@ const (
CAAAccountURI
// ProbeCTLogs enables HTTP probes to CT logs from the publisher
ProbeCTLogs
// SimplifiedVAHTTP enables the simplified VA http-01 rewrite that doesn't use
// a custom dialer.
SimplifiedVAHTTP
// HEAD requests to the WFE2 new-nonce endpoint should return HTTP StatusOK
// instead of HTTP StatusNoContent.
HeadNonceStatusOK

View File

@ -346,10 +346,10 @@ func (va *ValidationAuthorityImpl) setupHTTPValidation(
return dialer, record, nil
}
// fetchHTTPSimple invokes processHTTPValidation and if an error result is
// fetchHTTP invokes processHTTPValidation and if an error result is
// returned, converts it to a problem. Otherwise the results from
// processHTTPValidation are returned.
func (va *ValidationAuthorityImpl) fetchHTTPSimple(
func (va *ValidationAuthorityImpl) fetchHTTP(
ctx context.Context,
host string,
path string) ([]byte, []core.ValidationRecord, *probs.ProblemDetails) {

View File

@ -68,7 +68,7 @@ func TestPreresolvedDialerTimeout(t *testing.T) {
// If we get that, just retry until we get something other than "Network unreachable".
var prob *probs.ProblemDetails
for i := 0; i < 20; i++ {
_, _, prob = va.fetchHTTPSimple(ctx, "unroutable.invalid", "/.well-known/acme-challenge/whatever")
_, _, prob = va.fetchHTTP(ctx, "unroutable.invalid", "/.well-known/acme-challenge/whatever")
if prob != nil && strings.Contains(prob.Detail, "Network unreachable") {
continue
} else {
@ -586,7 +586,7 @@ func TestFallbackErr(t *testing.T) {
}
}
func TestFetchHTTPSimple(t *testing.T) {
func TestFetchHTTP(t *testing.T) {
// Create a test server
testSrv := httpTestSrv(t)
defer testSrv.Close()
@ -814,7 +814,7 @@ func TestFetchHTTPSimple(t *testing.T) {
t.Run(tc.Name, func(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*500)
defer cancel()
body, records, prob := va.fetchHTTPSimple(ctx, tc.Host, tc.Path)
body, records, prob := va.fetchHTTP(ctx, tc.Host, tc.Path)
if prob != nil && tc.ExpectedProblem == nil {
t.Errorf("expected nil prob, got %#v\n", prob)
} else if prob == nil && tc.ExpectedProblem != nil {

322
va/va.go
View File

@ -11,10 +11,7 @@ import (
"encoding/hex"
"errors"
"fmt"
"io"
"io/ioutil"
"net"
"net/http"
"net/url"
"os"
"strconv"
@ -232,101 +229,6 @@ func (va ValidationAuthorityImpl) getAddrs(ctx context.Context, hostname string)
return addrs, nil
}
type addrRecord struct {
used net.IP
tried []net.IP
}
// http01Dialer is a struct that exists to provide a dialer like object with
// a `DialContext` method that can be given to an http.Transport for HTTP-01
// validation. The primary purpose of the http01Dialer's DialContext method
// is to circumvent traditional DNS lookup and to use the IP addresses in the
// addr slice.
type http01Dialer struct {
addrs []net.IP
hostname string
port string
stats metrics.Scope
dialerCount int
timeout time.Duration
addrInfoChan chan addrRecord
}
// realDialer is used to create a true `net.Dialer` that can be used once an IP
// address to connect to is determined. It increments the `dialerCount` integer
// to track how many "fresh" dialer instances have been created during a
// `DialContext` for testing purposes.
func (d *http01Dialer) realDialer() *net.Dialer {
// Record that we created a new instance of a real net.Dialer
d.dialerCount++
return &net.Dialer{Timeout: d.timeout}
}
// DialContext processes the IP addresses from the inner validation record, using
// `realDialer` to make connections as required. For dual-homed hosts an initial
// IPv6 connection will be made followed by a IPv4 connection if there is a failure
// with the IPv6 connection.
func (d *http01Dialer) DialContext(ctx context.Context, _, _ string) (net.Conn, error) {
deadline, ok := ctx.Deadline()
if !ok {
// Shouldn't happen: All requests should have a deadline by this point.
deadline = time.Now().Add(100 * time.Second)
} else {
// Set the context deadline slightly shorter than the HTTP deadline, so we
// get the dial error rather than a generic "deadline exceeded" error. This
// lets us give a more specific error to the subscriber.
deadline = deadline.Add(-10 * time.Millisecond)
}
ctx, cancel := context.WithDeadline(ctx, deadline)
defer cancel()
var realDialer *net.Dialer
var addrInfo addrRecord
// Split the available addresses into v4 and v6 addresses
v4, v6 := availableAddresses(d.addrs)
// If there is at least one IPv6 address then try it first
if len(v6) > 0 {
address := net.JoinHostPort(v6[0].String(), d.port)
addrInfo.used = v6[0]
realDialer = d.realDialer()
conn, err := realDialer.DialContext(ctx, "tcp", address)
// If there is no error, return immediately
if err == nil {
d.addrInfoChan <- addrInfo
return conn, err
}
// Otherwise, we note that we tried an address and fall back to trying IPv4
addrInfo.tried = append(addrInfo.tried, addrInfo.used)
d.stats.Inc("IPv4Fallback", 1)
}
// If there are no IPv4 addresses and we tried an IPv6 address return an
// error - there's nothing left to try
if len(v4) == 0 && len(addrInfo.tried) > 0 {
d.addrInfoChan <- addrInfo
return nil,
fmt.Errorf("Unable to contact %q at %q, no IPv4 addresses to try as fallback",
d.hostname, addrInfo.tried[0])
} else if len(v4) == 0 && len(addrInfo.tried) == 0 {
// It shouldn't be possible that there are no IPv4 addresses and no previous
// attempts at an IPv6 address connection but be defensive about it anyway
d.addrInfoChan <- addrInfo
return nil, fmt.Errorf("no IP addresses found for %q", d.hostname)
}
// Otherwise if there are no IPv6 addresses, or there was an error
// talking to the first IPv6 address, try the first IPv4 address
addrInfo.used = v4[0]
d.addrInfoChan <- addrInfo
realDialer = d.realDialer()
return realDialer.DialContext(ctx, "tcp", net.JoinHostPort(v4[0].String(), d.port))
}
// availableAddresses takes a ValidationRecord and splits the AddressesResolved
// into a list of IPv4 and IPv6 addresses.
func availableAddresses(allAddrs []net.IP) (v4 []net.IP, v6 []net.IP) {
@ -340,221 +242,6 @@ func availableAddresses(allAddrs []net.IP) (v4 []net.IP, v6 []net.IP) {
return
}
// newHTTP01Dialer initializes a http01Dialer for the relevant hostname and port
// number
func (va *ValidationAuthorityImpl) newHTTP01Dialer(host string, port int, addrs []net.IP) http01Dialer {
return http01Dialer{
hostname: host,
port: strconv.Itoa(port),
addrs: addrs,
stats: va.stats,
timeout: va.singleDialTimeout,
addrInfoChan: make(chan addrRecord, 1),
}
}
// Validation methods
func (va *ValidationAuthorityImpl) fetchHTTP(ctx context.Context, identifier core.AcmeIdentifier, path string, useTLS bool, input core.Challenge) ([]byte, []core.ValidationRecord, *probs.ProblemDetails) {
challenge := input
host := identifier.Value
scheme := "http"
port := va.httpPort
if useTLS {
scheme = "https"
port = va.httpsPort
}
urlHost := host
if !((scheme == "http" && port == 80) ||
(scheme == "https" && port == 443)) {
urlHost = net.JoinHostPort(host, strconv.Itoa(port))
}
url := &url.URL{
Scheme: scheme,
Host: urlHost,
Path: path,
}
va.log.AuditInfof("Attempting to validate %s for %s", challenge.Type, url)
httpRequest, err := http.NewRequest("GET", url.String(), nil)
if err != nil {
va.log.Infof("Failed to parse URL '%s'. err=[%#v] errStr=[%s]", identifier, err, err)
return nil, nil, probs.Malformed("URL provided for HTTP was invalid")
}
httpRequest = httpRequest.WithContext(ctx)
if va.userAgent != "" {
httpRequest.Header["User-Agent"] = []string{va.userAgent}
}
// Build a base validation record that we will later populate with relevant IP
// addresses etc
baseRecord := core.ValidationRecord{
Hostname: host,
Port: strconv.Itoa(port),
URL: url.String(),
}
// Resolve IP addresses and construct custom dialer
addrs, prob := va.getAddrs(ctx, host)
if prob != nil {
return nil, []core.ValidationRecord{baseRecord}, prob
}
baseRecord.AddressesResolved = addrs
dialer := va.newHTTP01Dialer(host, port, addrs)
// Start with an empty validation record list - we will add a record after
// each dialer.DialContext()
var validationRecords []core.ValidationRecord
tr := &http.Transport{
// We are talking to a client that does not yet have a certificate,
// so we accept a temporary, invalid one.
TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
// We don't expect to make multiple requests to a client, so close
// connection immediately.
DisableKeepAlives: true,
// Intercept DialContext in order to connect to the IP address we
// select.
DialContext: dialer.DialContext,
// We don't want idle connections, but 0 means "unlimited," so we pick 1.
MaxIdleConns: 1,
IdleConnTimeout: time.Second,
TLSHandshakeTimeout: 10 * time.Second,
}
// Some of our users use mod_security. Mod_security sees a lack of Accept
// headers as bot behavior and rejects requests. While this is a bug in
// mod_security's rules (given that the HTTP specs disagree with that
// requirement), we add the Accept header now in order to fix our
// mod_security users' mysterious breakages. See
// <https://github.com/SpiderLabs/owasp-modsecurity-crs/issues/265> and
// <https://github.com/letsencrypt/boulder/issues/1019>. This was done
// because it's a one-line fix with no downside. We're not likely to want to
// do many more things to satisfy misunderstandings around HTTP.
httpRequest.Header.Set("Accept", "*/*")
numRedirects := 0
logRedirect := func(req *http.Request, via []*http.Request) error {
if numRedirects >= maxRedirect {
return fmt.Errorf("Too many redirects")
}
numRedirects++
// Set Accept header for mod_security (see the other place the header is
// set)
req.Header.Set("Accept", "*/*")
if va.userAgent != "" {
req.Header["User-Agent"] = []string{va.userAgent}
}
if req.URL.Scheme != "http" && req.URL.Scheme != "https" {
return berrors.ConnectionFailureError(
"Invalid protocol scheme in redirect target. "+
`Only "http" and "https" protocol schemes are supported, not %q`, req.URL.Scheme)
}
urlHost = req.URL.Host
reqHost := req.URL.Host
var reqPort int
if h, p, err := net.SplitHostPort(reqHost); err == nil {
reqHost = h
reqPort, err = strconv.Atoi(p)
if err != nil {
return err
}
if reqPort != va.httpPort && reqPort != va.httpsPort {
return berrors.ConnectionFailureError(
"Invalid port in redirect target. Only ports %d and %d are supported, not %d",
va.httpPort, va.httpsPort, reqPort)
}
} else if strings.ToLower(req.URL.Scheme) == "https" {
reqPort = va.httpsPort
} else {
reqPort = va.httpPort
}
// We do not want to redirect to any bare IP addresses. Only domain names
if net.ParseIP(reqHost) != nil {
return berrors.ConnectionFailureError(
"Invalid host in redirect target %q. "+
"Only domain names are supported, not IP addresses", reqHost)
}
// Since we've used dialer.DialContext we need to drain the address info
// channel and build a validation record using it and baseRecord so that
// we have a record for the host that sent the redirect.
addrInfo := <-dialer.addrInfoChan
record := baseRecord
record.AddressUsed, record.AddressesTried = addrInfo.used, addrInfo.tried
validationRecords = append(validationRecords, record)
// Update base record host, port, and URL for next dial. If there isn't
// another redirect this will be used by the parent scope to construct
// the final record.
baseRecord.Hostname = reqHost
baseRecord.Port = strconv.Itoa(reqPort)
baseRecord.URL = req.URL.String()
// Resolve new hostname and construct a new dialer
addrs, prob := va.getAddrs(ctx, reqHost)
if prob != nil {
// Since we won't call dialer.DialContext again the parent scope
// will block waiting for something from dialer.addrInfoChan so
// we put an empty addrRecord struct in the channel.
dialer.addrInfoChan <- addrRecord{}
return prob
}
baseRecord.AddressesResolved = addrs
dialer = va.newHTTP01Dialer(reqHost, reqPort, addrs)
tr.DialContext = dialer.DialContext
va.log.Debugf("%s [%s] redirect from %q to %q", challenge.Type, identifier,
via[len(via)-1].URL.String(), req.URL.String())
return nil
}
client := http.Client{
Transport: tr,
CheckRedirect: logRedirect,
}
httpResponse, err := client.Do(httpRequest)
// Read the address info from the dialer and update the base record with it,
// then append the it to the slice of records
addrInfo := <-dialer.addrInfoChan
baseRecord.AddressUsed, baseRecord.AddressesTried = addrInfo.used, addrInfo.tried
validationRecords = append(validationRecords, baseRecord)
if err != nil {
va.log.Infof("HTTP request to %s failed. err=[%#v] errStr=[%s]", url, err, err)
return nil, validationRecords, detailedError(err)
}
body, err := ioutil.ReadAll(&io.LimitedReader{R: httpResponse.Body, N: maxResponseSize})
closeErr := httpResponse.Body.Close()
if err == nil {
err = closeErr
}
if err != nil {
va.log.Infof("Error reading HTTP response body from %s. err=[%#v] errStr=[%s]", url, err, err)
return nil, validationRecords, probs.Unauthorized("Error reading HTTP response body: %v", err)
}
// io.LimitedReader will silently truncate a Reader so if the
// resulting payload is the same size as maxResponseSize fail
if len(body) >= maxResponseSize {
return nil, validationRecords, probs.Unauthorized("Invalid response from %s: %q", url,
replaceInvalidUTF8(body))
}
if httpResponse.StatusCode != 200 {
va.log.Infof("Non-200 status code from HTTP: %s returned %d", url, httpResponse.StatusCode)
return nil, validationRecords, probs.Unauthorized("Invalid response from %s [%s]: %d",
url, validationRecords[len(validationRecords)-1].AddressUsed, httpResponse.StatusCode)
}
return body, validationRecords, nil
}
// certNames collects up all of a certificate's subject names (Subject CN and
// Subject Alternate Names) and reduces them to a unique, sorted set, typically for an
// error message
@ -721,14 +408,7 @@ func (va *ValidationAuthorityImpl) validateHTTP01(ctx context.Context, identifie
// Perform the fetch
path := fmt.Sprintf(".well-known/acme-challenge/%s", challenge.Token)
var body []byte
var validationRecords []core.ValidationRecord
var prob *probs.ProblemDetails
if features.Enabled(features.SimplifiedVAHTTP) {
body, validationRecords, prob = va.fetchHTTPSimple(ctx, identifier.Value, "/"+path)
} else {
body, validationRecords, prob = va.fetchHTTP(ctx, identifier, path, false, challenge)
}
body, validationRecords, prob := va.fetchHTTP(ctx, identifier.Value, "/"+path)
if prob != nil {
return validationRecords, prob
}

View File

@ -33,7 +33,6 @@ import (
"github.com/letsencrypt/boulder/bdns"
"github.com/letsencrypt/boulder/cmd"
"github.com/letsencrypt/boulder/core"
"github.com/letsencrypt/boulder/features"
blog "github.com/letsencrypt/boulder/log"
"github.com/letsencrypt/boulder/metrics"
"github.com/letsencrypt/boulder/metrics/mock_metrics"
@ -346,7 +345,9 @@ func TestHTTP(t *testing.T) {
if prob != nil {
t.Fatalf("Failed to follow 301 redirect")
}
test.AssertEquals(t, len(log.GetAllMatching(`redirect from ".*/`+pathMoved+`" to ".*/`+pathValid+`"`)), 1)
redirectValid := `following redirect to host "" url "http://localhost/.well-known/acme-challenge/` + pathValid + `"`
matchedValidRedirect := log.GetAllMatching(redirectValid)
test.AssertEquals(t, len(matchedValidRedirect), 1)
log.Clear()
setChallengeToken(&chall, pathFound)
@ -354,8 +355,10 @@ func TestHTTP(t *testing.T) {
if prob != nil {
t.Fatalf("Failed to follow 302 redirect")
}
test.AssertEquals(t, len(log.GetAllMatching(`redirect from ".*/`+pathFound+`" to ".*/`+pathMoved+`"`)), 1)
test.AssertEquals(t, len(log.GetAllMatching(`redirect from ".*/`+pathMoved+`" to ".*/`+pathValid+`"`)), 1)
redirectMoved := `following redirect to host "" url "http://localhost/.well-known/acme-challenge/` + pathMoved + `"`
matchedMovedRedirect := log.GetAllMatching(redirectMoved)
test.AssertEquals(t, len(matchedValidRedirect), 1)
test.AssertEquals(t, len(matchedMovedRedirect), 1)
ipIdentifier := core.AcmeIdentifier{Type: core.IdentifierType("ip"), Value: "127.0.0.1"}
_, prob = va.validateHTTP01(ctx, ipIdentifier, chall)
@ -368,7 +371,7 @@ func TestHTTP(t *testing.T) {
if prob == nil {
t.Fatalf("Domain name is invalid.")
}
test.AssertEquals(t, prob.Type, probs.UnknownHostProblem)
test.AssertEquals(t, prob.Type, probs.ConnectionProblem)
}
func TestHTTPTimeout(t *testing.T) {
@ -381,56 +384,32 @@ func TestHTTPTimeout(t *testing.T) {
va, _ := setup(hs, 0)
setChallengeToken(&chall, pathWaitLong)
testCases := []struct {
Name string
SimplifiedVAHTTP bool
}{
{"Legacy VA HTTP", false},
{"Simplified VA HTTP", true},
expectMatch := regexp.MustCompile(
"Fetching http://localhost/.well-known/acme-challenge/wait-long: Timeout after connect")
started := time.Now()
timeout := 50 * time.Millisecond
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
_, prob := va.validateHTTP01(ctx, dnsi("localhost"), chall)
if prob == nil {
t.Fatalf("Connection should've timed out")
}
for _, tc := range testCases {
t.Run(tc.Name, func(t *testing.T) {
var expectMatch *regexp.Regexp
expectMatch = regexp.MustCompile(
"Fetching http://localhost:\\d+/.well-known/acme-challenge/wait-long: Timeout after connect")
took := time.Since(started)
// Check that the HTTP connection doesn't return before a timeout, and times
// out after the expected time
if took < timeout {
t.Fatalf("HTTP timed out before %s: %s with %s", timeout, took, prob)
}
if took > 2*timeout {
t.Fatalf("HTTP connection didn't timeout after %s", timeout)
}
test.AssertEquals(t, prob.Type, probs.ConnectionProblem)
if tc.SimplifiedVAHTTP {
err := features.Set(map[string]bool{"SimplifiedVAHTTP": true})
test.AssertNotError(t, err, "Failed to set SimplifiedVAHTTP feature flag")
defer features.Reset()
// Simplified VA HTTP error messages don't include the port number when
// it is equal to the va http port since it is implied by the `http://`
// protocol prefix.
expectMatch = regexp.MustCompile(
"Fetching http://localhost/.well-known/acme-challenge/wait-long: Timeout after connect")
}
started := time.Now()
timeout := 50 * time.Millisecond
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
_, prob := va.validateHTTP01(ctx, dnsi("localhost"), chall)
if prob == nil {
t.Fatalf("Connection should've timed out")
}
took := time.Since(started)
// Check that the HTTP connection doesn't return before a timeout, and times
// out after the expected time
if took < timeout {
t.Fatalf("HTTP timed out before %s: %s with %s", timeout, took, prob)
}
if took > 2*timeout {
t.Fatalf("HTTP connection didn't timeout after %s", timeout)
}
test.AssertEquals(t, prob.Type, probs.ConnectionProblem)
if !expectMatch.MatchString(prob.Detail) {
t.Errorf("Problem details incorrect. Got %q, expected to match %q",
prob.Detail, expectMatch)
}
})
if !expectMatch.MatchString(prob.Detail) {
t.Errorf("Problem details incorrect. Got %q, expected to match %q",
prob.Detail, expectMatch)
}
}
@ -504,7 +483,9 @@ func TestHTTPRedirectLookup(t *testing.T) {
if prob != nil {
t.Fatalf("Unexpected failure in redirect (%s): %s", pathMoved, prob)
}
test.AssertEquals(t, len(log.GetAllMatching(`redirect from ".*/`+pathMoved+`" to ".*/`+pathValid+`"`)), 1)
redirectValid := `following redirect to host "" url "http://localhost/.well-known/acme-challenge/` + pathValid + `"`
matchedValidRedirect := log.GetAllMatching(redirectValid)
test.AssertEquals(t, len(matchedValidRedirect), 1)
test.AssertEquals(t, len(log.GetAllMatching(`Resolved addresses for localhost: \[127.0.0.1\]`)), 2)
log.Clear()
@ -513,8 +494,9 @@ func TestHTTPRedirectLookup(t *testing.T) {
if prob != nil {
t.Fatalf("Unexpected failure in redirect (%s): %s", pathFound, prob)
}
test.AssertEquals(t, len(log.GetAllMatching(`redirect from ".*/`+pathFound+`" to ".*/`+pathMoved+`"`)), 1)
test.AssertEquals(t, len(log.GetAllMatching(`redirect from ".*/`+pathMoved+`" to ".*/`+pathValid+`"`)), 1)
redirectMoved := `following redirect to host "" url "http://localhost/.well-known/acme-challenge/` + pathMoved + `"`
matchedMovedRedirect := log.GetAllMatching(redirectMoved)
test.AssertEquals(t, len(matchedMovedRedirect), 1)
test.AssertEquals(t, len(log.GetAllMatching(`Resolved addresses for localhost: \[127.0.0.1\]`)), 3)
log.Clear()
@ -522,7 +504,7 @@ func TestHTTPRedirectLookup(t *testing.T) {
_, err := va.validateHTTP01(ctx, dnsi("localhost"), chall)
test.AssertError(t, err, chall.Token)
test.AssertEquals(t, len(log.GetAllMatching(`Resolved addresses for localhost: \[127.0.0.1\]`)), 1)
test.AssertEquals(t, len(log.GetAllMatching(`No valid IP addresses found for invalid.invalid`)), 1)
test.AssertDeepEquals(t, err, probs.ConnectionFailure("Fetching http://invalid.invalid/path: unknownHost :: No valid IP addresses found for invalid.invalid"))
log.Clear()
setChallengeToken(&chall, pathReLookup)
@ -530,7 +512,8 @@ func TestHTTPRedirectLookup(t *testing.T) {
if prob != nil {
t.Fatalf("Unexpected error in redirect (%s): %s", pathReLookup, prob)
}
test.AssertEquals(t, len(log.GetAllMatching(`redirect from ".*/`+pathReLookup+`" to ".*other.valid:\d+/path"`)), 1)
redirectPattern := `following redirect to host "" url "http://other.valid:\d+/path"`
test.AssertEquals(t, len(log.GetAllMatching(redirectPattern)), 1)
test.AssertEquals(t, len(log.GetAllMatching(`Resolved addresses for localhost: \[127.0.0.1\]`)), 1)
test.AssertEquals(t, len(log.GetAllMatching(`Resolved addresses for other.valid: \[127.0.0.1\]`)), 1)
@ -549,9 +532,10 @@ func TestHTTPRedirectLookup(t *testing.T) {
setChallengeToken(&chall, pathRedirectToFailingURL)
_, prob = va.validateHTTP01(ctx, dnsi("localhost"), chall)
test.AssertNotNil(t, prob, "Problem Details should not be nil")
test.AssertEquals(t, prob.Detail, fmt.Sprintf(
"Invalid response from http://localhost:%d/.well-known/acme-challenge/re-to-failing-url [127.0.0.1]: 500",
va.httpPort))
test.AssertDeepEquals(t, prob,
probs.Unauthorized(
fmt.Sprintf("Invalid response from http://other.valid:%d/500 [127.0.0.1]: 500",
va.httpPort)))
}
func TestHTTPRedirectLoop(t *testing.T) {
@ -1502,81 +1486,6 @@ func TestAvailableAddresses(t *testing.T) {
}
}
// TestHTTP01DialerFallback tests the underlying dialer used by HTTP01
// challenges. In particular it ensures that both the first IPv6 request and the
// subsequent IPv4 request get a new dialer each.
func TestHTTP01DialerFallback(t *testing.T) {
// Create a new challenge to use for the httpSrv
chall := core.HTTPChallenge01("")
setChallengeToken(&chall, core.NewToken())
// Create an IPv4 test server
hs := httpSrv(t, chall.Token)
defer hs.Close()
// Create a test VA
va, _ := setup(hs, 0)
// Create a test dialer for the dual homed host. There is only an IPv4 httpSrv
// so the IPv6 address returned in the AAAA record will always fail.
addrs, _ := va.getAddrs(context.Background(), "ipv4.and.ipv6.localhost")
d := va.newHTTP01Dialer("ipv4.and.ipv6.localhost", va.httpPort, addrs)
// Try to dial the dialer
_, dialProb := d.DialContext(context.Background(), "", "ipv4.and.ipv6.localhost")
// There shouldn't be a problem from this dial
test.AssertEquals(t, dialProb, nil)
// We should have constructed two inner dialers, one for each connection
test.AssertEquals(t, d.dialerCount, 2)
// We expect one validation record to be present
test.Assert(t, len(d.addrInfoChan) == 1, "there should be one address info struct in the dialer.addrInfoChan chan")
addrInfo := <-d.addrInfoChan
// We expect that the address used was the IPv4 localhost address
test.AssertEquals(t, addrInfo.used.String(), "127.0.0.1")
// We expect that one address was tried before the address used
test.AssertEquals(t, len(addrInfo.tried), 1)
// We expect that IPv6 address was tried before the address used
test.AssertEquals(t, addrInfo.tried[0].String(), "::1")
}
func TestFallbackDialer(t *testing.T) {
// Create a new challenge to use for the httpSrv
chall := core.HTTPChallenge01("")
setChallengeToken(&chall, core.NewToken())
// Create an IPv4 test server
hs := httpSrv(t, chall.Token)
defer hs.Close()
// Create a test VA
va, _ := setup(hs, 0)
ctrl := gomock.NewController(t)
defer ctrl.Finish()
scope := mock_metrics.NewMockScope(ctrl)
va.stats = scope
// We expect the IPV4 Fallback stat to be incremented
scope.EXPECT().Inc("IPv4Fallback", int64(1))
// The validation is expected to succeed even though the V6 server
// doesn't exist because we fallback to the IPv4 address.
ident := dnsi("ipv4.and.ipv6.localhost")
records, prob := va.validateChallenge(ctx, ident, chall)
test.Assert(t, prob == nil, "validation failed with IPv6 fallback to IPv4")
// We expect one validation record to be present
test.AssertEquals(t, len(records), 1)
// We expect that the address used was the IPv4 localhost address
test.AssertEquals(t, records[0].AddressUsed.String(), "127.0.0.1")
// We expect that one address was tried before the address used
test.AssertEquals(t, len(records[0].AddressesTried), 1)
// We expect that IPv6 address was tried before the address used
test.AssertEquals(t, records[0].AddressesTried[0].String(), "::1")
}
func TestFallbackTLS(t *testing.T) {
// Create a new challenge to use for the httpSrv
chall := createChallenge(core.ChallengeTypeTLSSNI01)