diff --git a/features/features.go b/features/features.go index efde340c0d8..90f7c7315c7 100644 --- a/features/features.go +++ b/features/features.go @@ -119,6 +119,13 @@ type Config struct { // unique "INSERT ... RETURNING" functionality. InsertAuthzsIndividually bool + // AutomaticallyPauseZombieClients configures the RA to automatically track + // limiter to be the authoritative source of rate limiting information for + // automatically pausing clients who systemically fail every validation + // attempt. When disabled, only manually paused accountID:identifier pairs + // will be rejected. + AutomaticallyPauseZombieClients bool + // IncrementRateLimits uses Redis' IncrBy, instead of Set, for rate limit // accounting. This catches and denies spikes of requests much more // reliably. diff --git a/ra/ra.go b/ra/ra.go index 10f343d0444..7fe7c5d916d 100644 --- a/ra/ra.go +++ b/ra/ra.go @@ -122,6 +122,7 @@ type RegistrationAuthorityImpl struct { orderAges *prometheus.HistogramVec inflightFinalizes prometheus.Gauge certCSRMismatch prometheus.Counter + pauseCounter *prometheus.CounterVec } var _ rapb.RegistrationAuthorityServer = (*RegistrationAuthorityImpl)(nil) @@ -241,6 +242,12 @@ func NewRegistrationAuthorityImpl( }) stats.MustRegister(certCSRMismatch) + pauseCounter := prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: "paused_pairs", + Help: "Number of times a pause operation is performed, labeled by paused=[bool], repaused=[bool], grace=[bool]", + }, []string{"paused", "repaused", "grace"}) + stats.MustRegister(pauseCounter) + issuersByNameID := make(map[issuance.NameID]*issuance.Certificate) for _, issuer := range issuers { issuersByNameID[issuer.NameID()] = issuer @@ -276,6 +283,7 @@ func NewRegistrationAuthorityImpl( orderAges: orderAges, inflightFinalizes: inflightFinalizes, certCSRMismatch: certCSRMismatch, + pauseCounter: pauseCounter, } return ra } @@ -1810,15 +1818,17 @@ func (ra *RegistrationAuthorityImpl) recordValidation(ctx context.Context, authI } // countFailedValidation increments the failed authorizations per domain per -// account rate limit. There is no reason to surface errors from this function -// to the Subscriber, spends against this limit are best effort. -func (ra *RegistrationAuthorityImpl) countFailedValidation(ctx context.Context, regId int64, name string) { +// account rate limit. If the AutomaticallyPauseZombieClients feature has been +// enabled, it also increments the failed authorizations for pausing per domain +// per account rate limit. There is no reason to surface errors from this +// function to the Subscriber, spends against this limit are best effort. +func (ra *RegistrationAuthorityImpl) countFailedValidation(ctx context.Context, regId int64, ident identifier.ACMEIdentifier) { if ra.limiter == nil || ra.txnBuilder == nil { // Limiter is disabled. return } - txn, err := ra.txnBuilder.FailedAuthorizationsPerDomainPerAccountSpendOnlyTransaction(regId, name) + txn, err := ra.txnBuilder.FailedAuthorizationsPerDomainPerAccountSpendOnlyTransaction(regId, ident.Value) if err != nil { ra.log.Warningf("building rate limit transaction for the %s rate limit: %s", ratelimits.FailedAuthorizationsPerDomainPerAccount, err) } @@ -1830,6 +1840,54 @@ func (ra *RegistrationAuthorityImpl) countFailedValidation(ctx context.Context, } ra.log.Warningf("spending against the %s rate limit: %s", ratelimits.FailedAuthorizationsPerDomainPerAccount, err) } + + if features.Get().AutomaticallyPauseZombieClients { + txn, err = ra.txnBuilder.FailedAuthorizationsForPausingPerDomainPerAccountTransaction(regId, ident.Value) + if err != nil { + ra.log.Warningf("building rate limit transaction for the %s rate limit: %s", ratelimits.FailedAuthorizationsForPausingPerDomainPerAccount, err) + } + + decision, err := ra.limiter.Spend(ctx, txn) + if err != nil { + if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { + return + } + ra.log.Warningf("spending against the %s rate limit: %s", ratelimits.FailedAuthorizationsForPausingPerDomainPerAccount, err) + } + + if decision.Result(ra.clk.Now()) != nil { + resp, err := ra.SA.PauseIdentifiers(ctx, &sapb.PauseRequest{ + RegistrationID: regId, + Identifiers: []*corepb.Identifier{ + { + Type: string(ident.Type), + Value: ident.Value, + }, + }, + }) + if err != nil { + ra.log.Warningf("failed to pause %d/%q: %s", regId, ident.Value, err) + } + ra.pauseCounter.With(prometheus.Labels{ + "paused": strconv.FormatBool(resp.Paused > 0), + "repaused": strconv.FormatBool(resp.Repaused > 0), + "grace": strconv.FormatBool(resp.Paused <= 0 && resp.Repaused <= 0), + }).Inc() + } + } +} + +// resetAccountPausingLimit resets bucket to maximum capacity for given account. +// There is no reason to surface errors from this function to the Subscriber. +func (ra *RegistrationAuthorityImpl) resetAccountPausingLimit(ctx context.Context, regId int64, ident identifier.ACMEIdentifier) { + bucketKey, err := ratelimits.NewRegIdDomainBucketKey(ratelimits.FailedAuthorizationsForPausingPerDomainPerAccount, regId, ident.Value) + if err != nil { + ra.log.Warningf("creating bucket key for regID=[%d] identifier=[%s]: %s", regId, ident.Value, err) + } + err = ra.limiter.Reset(ctx, bucketKey) + if err != nil { + ra.log.Warningf("resetting bucket for regID=[%d] identifier=[%s]: %s", regId, ident.Value, err) + } } // PerformValidation initiates validation for a specific challenge associated @@ -1953,9 +2011,12 @@ func (ra *RegistrationAuthorityImpl) PerformValidation( if prob != nil { challenge.Status = core.StatusInvalid challenge.Error = prob - go ra.countFailedValidation(vaCtx, authz.RegistrationID, authz.Identifier.Value) + go ra.countFailedValidation(vaCtx, authz.RegistrationID, authz.Identifier) } else { challenge.Status = core.StatusValid + if features.Get().AutomaticallyPauseZombieClients { + ra.resetAccountPausingLimit(vaCtx, authz.RegistrationID, authz.Identifier) + } } challenge.Validated = &vStart authz.Challenges[challIndex] = *challenge diff --git a/ra/ra_test.go b/ra/ra_test.go index ea77c22ecb9..7cb1a2cf842 100644 --- a/ra/ra_test.go +++ b/ra/ra_test.go @@ -48,6 +48,7 @@ import ( "github.com/letsencrypt/boulder/ctpolicy" "github.com/letsencrypt/boulder/ctpolicy/loglist" berrors "github.com/letsencrypt/boulder/errors" + "github.com/letsencrypt/boulder/features" "github.com/letsencrypt/boulder/goodkey" bgrpc "github.com/letsencrypt/boulder/grpc" "github.com/letsencrypt/boulder/identifier" @@ -277,7 +278,7 @@ func newAcctKey(t *testing.T) []byte { return acctKey } -func initAuthorities(t *testing.T) (*DummyValidationAuthority, sapb.StorageAuthorityClient, *RegistrationAuthorityImpl, clock.FakeClock, func()) { +func initAuthorities(t *testing.T) (*DummyValidationAuthority, sapb.StorageAuthorityClient, *RegistrationAuthorityImpl, *ratelimits.RedisSource, clock.FakeClock, func()) { err := json.Unmarshal(AccountKeyJSONA, &AccountKeyA) test.AssertNotError(t, err, "Failed to unmarshal public JWK") err = json.Unmarshal(AccountKeyJSONB, &AccountKeyB) @@ -348,6 +349,7 @@ func initAuthorities(t *testing.T) (*DummyValidationAuthority, sapb.StorageAutho }, }, nil, nil, 0, log, metrics.NoopRegisterer) + var source *ratelimits.RedisSource var limiter *ratelimits.Limiter var txnBuilder *ratelimits.TransactionBuilder if strings.Contains(os.Getenv("BOULDER_CONFIG_DIR"), "test/config-next") { @@ -371,8 +373,10 @@ func initAuthorities(t *testing.T) (*DummyValidationAuthority, sapb.StorageAutho } ring, err := bredis.NewRingFromConfig(rc, stats, log) test.AssertNotError(t, err, "making redis ring client") - source := ratelimits.NewRedisSource(ring.Ring, fc, stats) + source = ratelimits.NewRedisSource(ring.Ring, fc, stats) test.AssertNotNil(t, source, "source should not be nil") + err = source.Ping(context.Background()) + test.AssertNotError(t, err, "Ping should not error") limiter, err = ratelimits.NewLimiter(fc, source, stats) test.AssertNotError(t, err, "making limiter") txnBuilder, err = ratelimits.NewTransactionBuilder("../test/config-next/wfe2-ratelimit-defaults.yml", "") @@ -394,11 +398,11 @@ func initAuthorities(t *testing.T) (*DummyValidationAuthority, sapb.StorageAutho ra.CA = ca ra.OCSP = &mocks.MockOCSPGenerator{} ra.PA = pa - return va, sa, ra, fc, cleanUp + return va, sa, ra, source, fc, cleanUp } func TestValidateContacts(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ansible := "ansible:earth.sol.milkyway.laniakea/letsencrypt" @@ -475,7 +479,7 @@ func TestValidateContacts(t *testing.T) { } func TestNewRegistration(t *testing.T) { - _, sa, ra, _, cleanUp := initAuthorities(t) + _, sa, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() mailto := "mailto:foo@letsencrypt.org" acctKeyB, err := AccountKeyB.MarshalJSON() @@ -502,7 +506,7 @@ func TestNewRegistration(t *testing.T) { } func TestNewRegistrationContactsPresent(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() testCases := []struct { Name string @@ -574,7 +578,7 @@ func (sa *mockSAFailsNewRegistration) NewRegistration(_ context.Context, _ *core } func TestNewRegistrationSAFailure(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.SA = &mockSAFailsNewRegistration{} acctKeyB, err := AccountKeyB.MarshalJSON() @@ -592,7 +596,7 @@ func TestNewRegistrationSAFailure(t *testing.T) { } func TestNewRegistrationNoFieldOverwrite(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() mailto := "mailto:foo@letsencrypt.org" acctKeyC, err := AccountKeyC.MarshalJSON() @@ -614,7 +618,7 @@ func TestNewRegistrationNoFieldOverwrite(t *testing.T) { } func TestNewRegistrationBadKey(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() mailto := "mailto:foo@letsencrypt.org" shortKey, err := ShortKey.MarshalJSON() @@ -629,7 +633,7 @@ func TestNewRegistrationBadKey(t *testing.T) { } func TestRegistrationsPerIPOverrideUsage(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() regIP := net.ParseIP("4.5.6.7") @@ -687,7 +691,7 @@ func (sa *NoUpdateSA) UpdateRegistrationKey(_ context.Context, _ *sapb.UpdateReg } func TestUpdateRegistrationSame(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() mailto := "mailto:foo@letsencrypt.org" @@ -724,7 +728,7 @@ func TestUpdateRegistrationSame(t *testing.T) { } func TestPerformValidationExpired(t *testing.T) { - _, sa, ra, fc, cleanUp := initAuthorities(t) + _, sa, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() authz := createPendingAuthorization(t, sa, Identifier, fc.Now().Add(-2*time.Hour)) @@ -737,7 +741,7 @@ func TestPerformValidationExpired(t *testing.T) { } func TestPerformValidationAlreadyValid(t *testing.T) { - va, _, ra, _, cleanUp := initAuthorities(t) + va, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() // Create a finalized authorization @@ -782,7 +786,7 @@ func TestPerformValidationAlreadyValid(t *testing.T) { } func TestPerformValidationSuccess(t *testing.T) { - va, sa, ra, fc, cleanUp := initAuthorities(t) + va, sa, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() // We know this is OK because of TestNewAuthorization @@ -844,8 +848,317 @@ func TestPerformValidationSuccess(t *testing.T) { test.Assert(t, *challenge.Validated == expectedValidated, "Validated timestamp incorrect or missing") } +type mockSAPaused struct { + sync.RWMutex + sapb.StorageAuthorityClient + authorizationsForRegID map[int64]*corepb.Authorization + identifiersForRegID map[int64][]*corepb.Identifier + registrationsForRegID map[int64]*corepb.Registration +} + +func newMockSAPaused(sa sapb.StorageAuthorityClient) *mockSAPaused { + return &mockSAPaused{ + StorageAuthorityClient: sa, + authorizationsForRegID: make(map[int64]*corepb.Authorization), + identifiersForRegID: make(map[int64][]*corepb.Identifier), + registrationsForRegID: make(map[int64]*corepb.Registration), + } +} + +func (msa *mockSAPaused) GetRegistration(ctx context.Context, req *sapb.RegistrationID, _ ...grpc.CallOption) (*corepb.Registration, error) { + msa.Lock() + defer msa.Unlock() + regPB, ok := msa.registrationsForRegID[req.Id] + if !ok { + return nil, fmt.Errorf("Unable to find registration for regID %d", req.Id) + } + return regPB, nil +} + +func (msa *mockSAPaused) PauseIdentifiers(ctx context.Context, req *sapb.PauseRequest, _ ...grpc.CallOption) (*sapb.PauseIdentifiersResponse, error) { + msa.Lock() + defer msa.Unlock() + if len(req.Identifiers) <= 0 { + return nil, fmt.Errorf("No identifiers found to pause") + } + msa.identifiersForRegID[req.RegistrationID] = req.Identifiers + + counts := make(map[int64]int64) + for range msa.identifiersForRegID { + counts[req.RegistrationID]++ + } + return &sapb.PauseIdentifiersResponse{Paused: counts[req.RegistrationID]}, nil +} + +func (msa *mockSAPaused) GetPausedIdentifiers(ctx context.Context, req *sapb.RegistrationID, _ ...grpc.CallOption) (*sapb.Identifiers, error) { + msa.Lock() + defer msa.Unlock() + _, ok := msa.registrationsForRegID[req.Id] + if !ok { + return nil, fmt.Errorf("Unable to find registration for regID %d", req.Id) + } + idents, ok := msa.identifiersForRegID[req.Id] + if !ok { + return nil, fmt.Errorf("No identifiers paused yet") + } + return &sapb.Identifiers{Identifiers: idents}, nil +} + +func (msa *mockSAPaused) FinalizeAuthorization2(ctx context.Context, req *sapb.FinalizeAuthorizationRequest, _ ...grpc.CallOption) (*emptypb.Empty, error) { + return &emptypb.Empty{}, nil +} + +func TestPerformValidation_FailedValidationsTriggerPauseIdentifiersRatelimit(t *testing.T) { + if !strings.Contains(os.Getenv("BOULDER_CONFIG_DIR"), "test/config-next") { + t.Skip() + } + + va, sa, ra, redisSrc, fc, cleanUp := initAuthorities(t) + defer cleanUp() + + features.Set(features.Config{AutomaticallyPauseZombieClients: true}) + defer features.Reset() + + mockSA := newMockSAPaused(sa) + ra.SA = mockSA + + // Override the default ratelimits to only allow one failed validation. + txnBuilder, err := ratelimits.NewTransactionBuilder("testdata/one-failed-validation-before-pausing.yml", "") + test.AssertNotError(t, err, "making transaction composer") + ra.txnBuilder = txnBuilder + + // We know this is OK because of TestNewAuthorization + domain := "example.net" + authzPB := createPendingAuthorization(t, sa, domain, fc.Now().Add(12*time.Hour)) + mockSA.registrationsForRegID[authzPB.RegistrationID] = Registration + mockSA.authorizationsForRegID[authzPB.RegistrationID] = authzPB + + // We induce the bad path by setting a problem. This will consume all + // available capacity in the rate limit bucket. + va.PerformValidationRequestResultReturn = &vapb.ValidationResult{ + Records: []*corepb.ValidationRecord{ + { + AddressUsed: []byte("192.168.0.1"), + Hostname: domain, + Port: "8080", + Url: fmt.Sprintf("http://%s/", domain), + ResolverAddrs: []string{"rebound"}, + }, + }, + Problems: &corepb.ProblemDetails{ + Detail: fmt.Sprintf("CAA invalid for %s", domain), + }, + } + + challIdx := dnsChallIdx(t, authzPB.Challenges) + authzPB, err = ra.PerformValidation(ctx, &rapb.PerformValidationRequest{ + Authz: authzPB, + ChallengeIndex: challIdx, + }) + test.AssertNotError(t, err, "PerformValidation failed") + + select { + case r := <-va.performValidationRequest: + _ = r + case <-time.After(time.Second): + t.Fatal("Timed out waiting for DummyValidationAuthority.PerformValidation to complete") + } + + // Sleep so the RA has a chance to write to the SA + time.Sleep(100 * time.Millisecond) + + got, err := ra.SA.GetPausedIdentifiers(ctx, &sapb.RegistrationID{Id: authzPB.RegistrationID}, nil) + test.AssertError(t, err, "Should not have any paused identifiers yet, but found some") + test.AssertBoxedNil(t, got, "Should have received nil response, but did not") + test.AssertMetricWithLabelsEquals(t, ra.pauseCounter, prometheus.Labels{"paused": "false", "repaused": "false", "grace": "false"}, 0) + + // We need the bucket key to scan for in Redis + bucketKey, err := ratelimits.NewRegIdDomainBucketKey(ratelimits.FailedAuthorizationsForPausingPerDomainPerAccount, authzPB.RegistrationID, domain) + test.AssertNotError(t, err, "Should have been able to construct bucket key, but could not") + + // Verify that a redis entry exists for this accountID:identifier. + tat, err := redisSrc.Get(ctx, bucketKey) + test.AssertNotError(t, err, "Should not have errored, but did") + + // There is no more capacity and the next failed validation will effectively + // pause issuance attempts. The ratelimit file is written to increment + // capacity every 24 hours, so we can check that the TAT states that, not + // that it particularly matters in this context. + test.AssertEquals(t, tat, fc.Now().Add(24*time.Hour)) + + // A second failed validation should result in the identifier being paused + // due to the strict ratelimit. + va.PerformValidationRequestResultReturn = &vapb.ValidationResult{ + Records: []*corepb.ValidationRecord{ + { + AddressUsed: []byte("192.168.0.1"), + Hostname: domain, + Port: "8080", + Url: fmt.Sprintf("http://%s/", domain), + ResolverAddrs: []string{"rebound"}, + }, + }, + Problems: &corepb.ProblemDetails{ + Detail: fmt.Sprintf("CAA invalid for %s", domain), + }, + } + + challIdx = dnsChallIdx(t, authzPB.Challenges) + authzPB, err = ra.PerformValidation(ctx, &rapb.PerformValidationRequest{ + Authz: authzPB, + ChallengeIndex: challIdx, + }) + test.AssertNotError(t, err, "PerformValidation failed") + + select { + case r := <-va.performValidationRequest: + _ = r + case <-time.After(time.Second): + t.Fatal("Timed out waiting for DummyValidationAuthority.PerformValidation to complete") + } + + // Sleep so the RA has a chance to write to the SA + time.Sleep(100 * time.Millisecond) + + // Ensure the identifier:account:domain we expect to be paused actually is. + got, err = ra.SA.GetPausedIdentifiers(ctx, &sapb.RegistrationID{Id: authzPB.RegistrationID}, nil) + test.AssertNotError(t, err, "Should not have errored getting paused identifiers") + test.AssertEquals(t, len(got.Identifiers), 1) + test.AssertEquals(t, got.Identifiers[0].Value, domain) + test.AssertMetricWithLabelsEquals(t, ra.pauseCounter, prometheus.Labels{"paused": "true", "repaused": "false", "grace": "false"}, 1) + + err = ra.limiter.Reset(ctx, bucketKey) + test.AssertNotError(t, err, "Failed cleaning up redis") +} + +func TestPerformValidation_FailedThenSuccessfulValidationResetsPauseIdentifiersRatelimit(t *testing.T) { + if !strings.Contains(os.Getenv("BOULDER_CONFIG_DIR"), "test/config-next") { + t.Skip() + } + + va, sa, ra, redisSrc, fc, cleanUp := initAuthorities(t) + defer cleanUp() + + features.Set(features.Config{AutomaticallyPauseZombieClients: true}) + defer features.Reset() + + mockSA := newMockSAPaused(sa) + ra.SA = mockSA + + // Override the default ratelimits to only allow one failed validation. + txnBuilder, err := ratelimits.NewTransactionBuilder("testdata/two-failed-validations-before-pausing.yml", "") + test.AssertNotError(t, err, "making transaction composer") + ra.txnBuilder = txnBuilder + + // We know this is OK because of TestNewAuthorization + domain := "example.net" + authzPB := createPendingAuthorization(t, sa, "example.net", fc.Now().Add(12*time.Hour)) + mockSA.registrationsForRegID[authzPB.RegistrationID] = Registration + mockSA.authorizationsForRegID[authzPB.RegistrationID] = authzPB + + // We induce the bad path by setting a problem. This will consume all + // available capacity in the rate limit bucket. + va.PerformValidationRequestResultReturn = &vapb.ValidationResult{ + Records: []*corepb.ValidationRecord{ + { + AddressUsed: []byte("192.168.0.1"), + Hostname: domain, + Port: "8080", + Url: fmt.Sprintf("http://%s/", domain), + ResolverAddrs: []string{"rebound"}, + }, + }, + Problems: &corepb.ProblemDetails{ + Detail: fmt.Sprintf("CAA invalid for %s", domain), + }, + } + + challIdx := dnsChallIdx(t, authzPB.Challenges) + authzPB, err = ra.PerformValidation(ctx, &rapb.PerformValidationRequest{ + Authz: authzPB, + ChallengeIndex: challIdx, + }) + test.AssertNotError(t, err, "PerformValidation failed") + + select { + case r := <-va.performValidationRequest: + _ = r + case <-time.After(time.Second): + t.Fatal("Timed out waiting for DummyValidationAuthority.PerformValidation to complete") + } + + // Sleep so the RA has a chance to write to the SA + time.Sleep(100 * time.Millisecond) + + got, err := ra.SA.GetPausedIdentifiers(ctx, &sapb.RegistrationID{Id: authzPB.RegistrationID}, nil) + test.AssertError(t, err, "Should not have any paused identifiers yet, but found some") + test.AssertBoxedNil(t, got, "Should have received nil response, but did not") + test.AssertMetricWithLabelsEquals(t, ra.pauseCounter, prometheus.Labels{"paused": "false", "repaused": "false", "grace": "false"}, 0) + + // We need the bucket key to scan for in Redis + bucketKey, err := ratelimits.NewRegIdDomainBucketKey(ratelimits.FailedAuthorizationsForPausingPerDomainPerAccount, authzPB.RegistrationID, domain) + test.AssertNotError(t, err, "Should have been able to construct bucket key, but could not") + + // Verify that a redis entry exists for this accountID:identifier + tat, err := redisSrc.Get(ctx, bucketKey) + test.AssertNotError(t, err, "Should not have errored, but did") + + // We should have capacity for 1 more failed validation, the next TAT should + // be immediately (despite the fact that this clearly says now + 12 hours). + test.AssertEquals(t, tat, fc.Now().Add(12*time.Hour)) + + // + // Now the goal is to perform a successful validation which should reset the + // FailedAuthorizationsForPausingPerDomainPerAccount ratelimit. + // + + // We know this is OK because of TestNewAuthorization + authzPB = createPendingAuthorization(t, sa, domain, fc.Now().Add(12*time.Hour)) + + va.PerformValidationRequestResultReturn = &vapb.ValidationResult{ + Records: []*corepb.ValidationRecord{ + { + AddressUsed: []byte("192.168.0.1"), + Hostname: domain, + Port: "8080", + Url: fmt.Sprintf("http://%s/", domain), + ResolverAddrs: []string{"rebound"}, + }, + }, + Problems: nil, + } + + challIdx = dnsChallIdx(t, authzPB.Challenges) + authzPB, err = ra.PerformValidation(ctx, &rapb.PerformValidationRequest{ + Authz: authzPB, + ChallengeIndex: challIdx, + }) + test.AssertNotError(t, err, "PerformValidation failed") + mockSA.authorizationsForRegID[authzPB.RegistrationID] = authzPB + + select { + case r := <-va.performValidationRequest: + _ = r + case <-time.After(time.Second): + t.Fatal("Timed out waiting for DummyValidationAuthority.PerformValidation to complete") + } + + // We need the bucket key to scan for in Redis + bucketKey, err = ratelimits.NewRegIdDomainBucketKey(ratelimits.FailedAuthorizationsForPausingPerDomainPerAccount, authzPB.RegistrationID, domain) + test.AssertNotError(t, err, "Should have been able to construct bucket key, but could not") + + // Verify that the bucket no longer exists (because the limiter reset has + // deleted it). This indicates the accountID:identifier bucket has regained + // capacity avoiding being inadvertently paused. + _, err = redisSrc.Get(ctx, bucketKey) + test.AssertErrorIs(t, err, ratelimits.ErrBucketNotFound) + + err = ra.limiter.Reset(ctx, bucketKey) + test.AssertNotError(t, err, "Failed cleaning up redis") +} + func TestPerformValidationVAError(t *testing.T) { - va, sa, ra, fc, cleanUp := initAuthorities(t) + va, sa, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() authzPB := createPendingAuthorization(t, sa, Identifier, fc.Now().Add(12*time.Hour)) @@ -892,7 +1205,7 @@ func TestPerformValidationVAError(t *testing.T) { } func TestCertificateKeyNotEqualAccountKey(t *testing.T) { - _, sa, ra, _, cleanUp := initAuthorities(t) + _, sa, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() exp := ra.clk.Now().Add(365 * 24 * time.Hour) @@ -931,7 +1244,7 @@ func TestCertificateKeyNotEqualAccountKey(t *testing.T) { } func TestNewOrderRateLimiting(t *testing.T) { - _, _, ra, fc, cleanUp := initAuthorities(t) + _, _, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = 5 * 24 * time.Hour @@ -982,7 +1295,7 @@ func TestNewOrderRateLimiting(t *testing.T) { // TestEarlyOrderRateLimiting tests that NewOrder applies the certificates per // name/per FQDN rate limits against the order names. func TestEarlyOrderRateLimiting(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = 5 * 24 * time.Hour @@ -1044,7 +1357,7 @@ func (sa *mockInvalidAuthorizationsAuthority) CountInvalidAuthorizations2(ctx co } func TestAuthzFailedRateLimitingNewOrder(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.rlPolicies = &dummyRateLimitConfig{ @@ -1094,7 +1407,7 @@ func (m *mockSAWithNameCounts) FQDNSetExists(ctx context.Context, req *sapb.FQDN } func TestCheckCertificatesPerNameLimit(t *testing.T) { - _, _, ra, fc, cleanUp := initAuthorities(t) + _, _, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() rlp := ratelimit.RateLimitPolicy{ @@ -1184,7 +1497,7 @@ func TestCheckCertificatesPerNameLimit(t *testing.T) { // TestCheckExactCertificateLimit tests that the duplicate certificate limit // applied to FQDN sets is respected. func TestCheckExactCertificateLimit(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() // Create a rate limit with a small threshold @@ -1434,7 +1747,7 @@ func (m mockSAWithFQDNSet) FQDNSetTimestampsForWindow(_ context.Context, req *sa // without the feature flag for the fix enabled. // See https://github.com/letsencrypt/boulder/issues/2681 func TestExactPublicSuffixCertLimit(t *testing.T) { - _, _, ra, fc, cleanUp := initAuthorities(t) + _, _, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() // Simple policy that only allows 2 certificates per name. @@ -1485,7 +1798,7 @@ func TestExactPublicSuffixCertLimit(t *testing.T) { } func TestDeactivateAuthorization(t *testing.T) { - _, sa, ra, _, cleanUp := initAuthorities(t) + _, sa, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() exp := ra.clk.Now().Add(365 * 24 * time.Hour) @@ -1499,7 +1812,7 @@ func TestDeactivateAuthorization(t *testing.T) { } func TestDeactivateRegistration(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() // Deactivate failure because incomplete registration provided @@ -1554,7 +1867,7 @@ func (cr *caaRecorder) IsCAAValid( // Test that the right set of domain names have their CAA rechecked, based on // their `Validated` (attemptedAt in the database) timestamp. func TestRecheckCAADates(t *testing.T) { - _, _, ra, fc, cleanUp := initAuthorities(t) + _, _, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() recorder := &caaRecorder{names: make(map[string]bool)} ra.caa = recorder @@ -1744,7 +2057,7 @@ func (cf *caaFailer) IsCAAValid( } func TestRecheckCAAEmpty(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() err := ra.recheckCAA(context.Background(), nil) test.AssertNotError(t, err, "expected nil") @@ -1758,7 +2071,7 @@ func makeHTTP01Authorization(domain string) *core.Authorization { } func TestRecheckCAASuccess(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() authzs := []*core.Authorization{ makeHTTP01Authorization("a.com"), @@ -1770,7 +2083,7 @@ func TestRecheckCAASuccess(t *testing.T) { } func TestRecheckCAAFail(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.caa = &caaFailer{} authzs := []*core.Authorization{ @@ -1821,7 +2134,7 @@ func TestRecheckCAAFail(t *testing.T) { } func TestRecheckCAAInternalServerError(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.caa = &caaFailer{} authzs := []*core.Authorization{ @@ -1835,7 +2148,7 @@ func TestRecheckCAAInternalServerError(t *testing.T) { } func TestNewOrder(t *testing.T) { - _, _, ra, fc, cleanUp := initAuthorities(t) + _, _, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = time.Hour @@ -1904,7 +2217,7 @@ func TestNewOrder(t *testing.T) { // an identical order results in only one order being created & subsequently // reused. func TestNewOrderReuse(t *testing.T) { - _, _, ra, fc, cleanUp := initAuthorities(t) + _, _, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() ctx := context.Background() @@ -2013,7 +2326,7 @@ func TestNewOrderReuse(t *testing.T) { } func TestNewOrderReuseInvalidAuthz(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ctx := context.Background() @@ -2074,7 +2387,7 @@ func (mock *mockSACountPendingFails) CountPendingAuthorizations2(ctx context.Con // Ensure that we don't bother to call the SA to count pending authorizations // when an "unlimited" limit is set. func TestPendingAuthorizationsUnlimited(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.rlPolicies = &dummyRateLimitConfig{ @@ -2111,7 +2424,7 @@ func (sa *mockInvalidPlusValidAuthzAuthority) CountInvalidAuthorizations2(ctx co // Test that the failed authorizations limit is checked before authz reuse. func TestNewOrderCheckFailedAuthorizationsFirst(t *testing.T) { - _, _, ra, clk, cleanUp := initAuthorities(t) + _, _, ra, _, clk, cleanUp := initAuthorities(t) defer cleanUp() // Create an order (and thus a pending authz) for example.com @@ -2238,7 +2551,7 @@ func (msa *mockSAWithAuthzs) NewOrderAndAuthzs(ctx context.Context, req *sapb.Ne // for background - this safety check was previously broken! // https://github.com/letsencrypt/boulder/issues/3420 func TestNewOrderAuthzReuseSafety(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ctx := context.Background() @@ -2314,7 +2627,7 @@ func TestNewOrderAuthzReuseSafety(t *testing.T) { } func TestNewOrderWildcard(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = time.Hour @@ -2479,7 +2792,7 @@ func TestNewOrderWildcard(t *testing.T) { } func TestNewOrderExpiry(t *testing.T) { - _, _, ra, clk, cleanUp := initAuthorities(t) + _, _, ra, _, clk, cleanUp := initAuthorities(t) defer cleanUp() ctx := context.Background() @@ -2547,7 +2860,7 @@ func TestNewOrderExpiry(t *testing.T) { } func TestFinalizeOrder(t *testing.T) { - _, sa, ra, fc, cleanUp := initAuthorities(t) + _, sa, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = time.Hour @@ -2833,7 +3146,7 @@ func TestFinalizeOrder(t *testing.T) { } func TestFinalizeOrderWithMixedSANAndCN(t *testing.T) { - _, sa, ra, _, cleanUp := initAuthorities(t) + _, sa, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = time.Hour @@ -2895,7 +3208,7 @@ func TestFinalizeOrderWithMixedSANAndCN(t *testing.T) { } func TestFinalizeOrderWildcard(t *testing.T) { - _, sa, ra, _, cleanUp := initAuthorities(t) + _, sa, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() // Pick an expiry in the future @@ -2997,7 +3310,7 @@ func TestFinalizeOrderWildcard(t *testing.T) { } func TestFinalizeOrderDisabledChallenge(t *testing.T) { - _, sa, ra, fc, cleanUp := initAuthorities(t) + _, sa, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() // Create a random domain @@ -3052,7 +3365,7 @@ func TestFinalizeOrderDisabledChallenge(t *testing.T) { } func TestIssueCertificateAuditLog(t *testing.T) { - _, sa, ra, _, cleanUp := initAuthorities(t) + _, sa, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() // Set up order and authz expiries @@ -3175,7 +3488,7 @@ func TestIssueCertificateAuditLog(t *testing.T) { } func TestIssueCertificateCAACheckLog(t *testing.T) { - _, sa, ra, fc, cleanUp := initAuthorities(t) + _, sa, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() // Set up order and authz expiries. @@ -3286,7 +3599,7 @@ func TestIssueCertificateCAACheckLog(t *testing.T) { // // See https://github.com/letsencrypt/boulder/issues/3201 func TestUpdateMissingAuthorization(t *testing.T) { - _, sa, ra, fc, cleanUp := initAuthorities(t) + _, sa, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() ctx := context.Background() @@ -3306,7 +3619,7 @@ func TestUpdateMissingAuthorization(t *testing.T) { } func TestPerformValidationBadChallengeType(t *testing.T) { - _, _, ra, fc, cleanUp := initAuthorities(t) + _, _, ra, _, fc, cleanUp := initAuthorities(t) defer cleanUp() pa, err := policy.New(map[core.AcmeChallenge]bool{}, blog.NewMock()) test.AssertNotError(t, err, "Couldn't create PA") @@ -3346,7 +3659,7 @@ func (mp *timeoutPub) SubmitToSingleCTWithResult(_ context.Context, _ *pubpb.Req } func TestCTPolicyMeasurements(t *testing.T) { - _, ssa, ra, _, cleanup := initAuthorities(t) + _, ssa, ra, _, _, cleanup := initAuthorities(t) defer cleanup() ra.ctpolicy = ctpolicy.New(&timeoutPub{}, loglist.List{ @@ -3479,7 +3792,7 @@ func (ca *mockCAFailCertForPrecert) IssueCertificateForPrecertificate( // `ra.issueCertificateInner` are propagated correctly, with the part of the // issuance process that failed prefixed on the error message. func TestIssueCertificateInnerErrs(t *testing.T) { - _, sa, ra, _, cleanUp := initAuthorities(t) + _, sa, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = 24 * time.Hour @@ -3612,7 +3925,7 @@ func (sa *mockSAWithFinalize) FQDNSetExists(ctx context.Context, in *sapb.FQDNSe } func TestIssueCertificateInnerWithProfile(t *testing.T) { - _, _, ra, fc, cleanup := initAuthorities(t) + _, _, ra, _, fc, cleanup := initAuthorities(t) defer cleanup() // Generate a reasonable-looking CSR and cert to pass the matchesCSR check. @@ -3648,7 +3961,7 @@ func TestIssueCertificateInnerWithProfile(t *testing.T) { } func TestIssueCertificateOuter(t *testing.T) { - _, sa, ra, fc, cleanup := initAuthorities(t) + _, sa, ra, _, fc, cleanup := initAuthorities(t) defer cleanup() ra.orderLifetime = 24 * time.Hour @@ -3702,7 +4015,7 @@ func TestIssueCertificateOuter(t *testing.T) { } func TestNewOrderMaxNames(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.maxNames = 2 @@ -3897,7 +4210,7 @@ func (msgo *mockSAGenerateOCSP) GetCertificateStatus(_ context.Context, req *sap } func TestGenerateOCSP(t *testing.T) { - _, _, ra, clk, cleanUp := initAuthorities(t) + _, _, ra, _, clk, cleanUp := initAuthorities(t) defer cleanUp() ra.OCSP = &mockOCSPA{} @@ -3937,7 +4250,7 @@ func (msgo *mockSALongExpiredSerial) GetSerialMetadata(_ context.Context, req *s } func TestGenerateOCSPLongExpiredSerial(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.OCSP = &mockOCSPA{} @@ -3965,7 +4278,7 @@ func (msgo *mockSAUnknownSerial) GetSerialMetadata(_ context.Context, req *sapb. } func TestGenerateOCSPUnknownSerial(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.OCSP = &mockOCSPA{} @@ -3983,7 +4296,7 @@ func TestGenerateOCSPUnknownSerial(t *testing.T) { } func TestRevokeCertByApplicant_Subscriber(t *testing.T) { - _, _, ra, clk, cleanUp := initAuthorities(t) + _, _, ra, _, clk, cleanUp := initAuthorities(t) defer cleanUp() ra.OCSP = &mockOCSPA{} @@ -4058,7 +4371,7 @@ func (msa *mockSARevocationWithAuthzs) GetValidAuthorizations2(ctx context.Conte } func TestRevokeCertByApplicant_Controller(t *testing.T) { - _, _, ra, clk, cleanUp := initAuthorities(t) + _, _, ra, _, clk, cleanUp := initAuthorities(t) defer cleanUp() ra.OCSP = &mockOCSPA{} @@ -4099,7 +4412,7 @@ func TestRevokeCertByApplicant_Controller(t *testing.T) { } func TestRevokeCertByKey(t *testing.T) { - _, _, ra, clk, cleanUp := initAuthorities(t) + _, _, ra, _, clk, cleanUp := initAuthorities(t) defer cleanUp() ra.OCSP = &mockOCSPA{} @@ -4151,7 +4464,7 @@ func TestRevokeCertByKey(t *testing.T) { } func TestAdministrativelyRevokeCertificate(t *testing.T) { - _, _, ra, clk, cleanUp := initAuthorities(t) + _, _, ra, _, clk, cleanUp := initAuthorities(t) defer cleanUp() ra.OCSP = &mockOCSPA{} @@ -4296,7 +4609,7 @@ func TestAdministrativelyRevokeCertificate(t *testing.T) { } func TestNewOrderRateLimitingExempt(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() ra.orderLifetime = 5 * 24 * time.Hour @@ -4334,7 +4647,7 @@ func TestNewOrderRateLimitingExempt(t *testing.T) { } func TestNewOrderFailedAuthzRateLimitingExempt(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() exampleOrder := &rapb.NewOrderRequest{ @@ -4395,7 +4708,7 @@ func (sa *mockNewOrderMustBeReplacementAuthority) NewOrderAndAuthzs(ctx context. } func TestNewOrderReplacesSerialCarriesThroughToSA(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() exampleOrder := &rapb.NewOrderRequest{ @@ -4431,7 +4744,7 @@ func (sa *mockSAUnpauseAccount) UnpauseAccount(_ context.Context, req *sapb.Regi // the requested RegID to the SA, and correctly passes the SA's count back to // the caller. func TestUnpauseAccount(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() mockSA := mockSAUnpauseAccount{identsToUnpause: 0} @@ -4453,7 +4766,7 @@ func TestUnpauseAccount(t *testing.T) { } func TestGetAuthorization(t *testing.T) { - _, _, ra, _, cleanup := initAuthorities(t) + _, _, ra, _, _, cleanup := initAuthorities(t) defer cleanup() ra.SA = &mockSAWithAuthzs{ @@ -4532,7 +4845,7 @@ func (sa *mockSARecordingRegistration) UpdateRegistrationKey(ctx context.Context // to the SA; passes the updated Registration back to the caller; and can return // an error. func TestUpdateRegistrationContact(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() expectRegID := int64(1) @@ -4586,7 +4899,7 @@ func TestUpdateRegistrationContact(t *testing.T) { // correctly requires a registration ID and key, passes them to the SA, and // passes the updated Registration back to the caller. func TestUpdateRegistrationKey(t *testing.T) { - _, _, ra, _, cleanUp := initAuthorities(t) + _, _, ra, _, _, cleanUp := initAuthorities(t) defer cleanUp() expectRegID := int64(1) diff --git a/ra/testdata/one-failed-validation-before-pausing.yml b/ra/testdata/one-failed-validation-before-pausing.yml new file mode 100644 index 00000000000..57bf710666f --- /dev/null +++ b/ra/testdata/one-failed-validation-before-pausing.yml @@ -0,0 +1,4 @@ +FailedAuthorizationsForPausingPerDomainPerAccount: + count: 1 + burst: 1 + period: 24h diff --git a/ra/testdata/two-failed-validations-before-pausing.yml b/ra/testdata/two-failed-validations-before-pausing.yml new file mode 100644 index 00000000000..5024aad5c0d --- /dev/null +++ b/ra/testdata/two-failed-validations-before-pausing.yml @@ -0,0 +1,4 @@ +FailedAuthorizationsForPausingPerDomainPerAccount: + count: 2 + burst: 1 + period: 24h diff --git a/ratelimits/limiter.go b/ratelimits/limiter.go index 83d6752ed1f..86aa3cfae6e 100644 --- a/ratelimits/limiter.go +++ b/ratelimits/limiter.go @@ -111,6 +111,8 @@ func (d *Decision) Result(now time.Time) error { retryAfter := d.retryIn + jitter retryAfterTs := now.UTC().Add(retryAfter).Format("2006-01-02 15:04:05 MST") + // There is no case for FailedAuthorizationsForPausingPerDomainPerAccount + // because the RA will pause clients who exceed that ratelimit. switch d.transaction.limit.name { case NewRegistrationsPerIPAddress: return berrors.RegistrationsPerIPAddressError( diff --git a/ratelimits/names.go b/ratelimits/names.go index 4495f863865..99221ae0cec 100644 --- a/ratelimits/names.go +++ b/ratelimits/names.go @@ -76,18 +76,29 @@ const ( // Note: When this is referenced in an overrides file, the fqdnSet MUST be // passed as a comma-separated list of domain names. CertificatesPerFQDNSet + + // FailedAuthorizationsForPausingPerDomainPerAccount is similar to + // FailedAuthorizationsPerDomainPerAccount in that it uses two different + // bucket keys depending on the context: + // - When referenced in an overrides file: uses bucket key 'enum:regId', + // where regId is the ACME registration Id of the account. + // - When referenced in a transaction: uses bucket key 'enum:regId:domain', + // where regId is the ACME registration Id of the account and domain is a + // domain name in the certificate. + FailedAuthorizationsForPausingPerDomainPerAccount ) // nameToString is a map of Name values to string names. var nameToString = map[Name]string{ - Unknown: "Unknown", - NewRegistrationsPerIPAddress: "NewRegistrationsPerIPAddress", - NewRegistrationsPerIPv6Range: "NewRegistrationsPerIPv6Range", - NewOrdersPerAccount: "NewOrdersPerAccount", - FailedAuthorizationsPerDomainPerAccount: "FailedAuthorizationsPerDomainPerAccount", - CertificatesPerDomain: "CertificatesPerDomain", - CertificatesPerDomainPerAccount: "CertificatesPerDomainPerAccount", - CertificatesPerFQDNSet: "CertificatesPerFQDNSet", + Unknown: "Unknown", + NewRegistrationsPerIPAddress: "NewRegistrationsPerIPAddress", + NewRegistrationsPerIPv6Range: "NewRegistrationsPerIPv6Range", + NewOrdersPerAccount: "NewOrdersPerAccount", + FailedAuthorizationsPerDomainPerAccount: "FailedAuthorizationsPerDomainPerAccount", + CertificatesPerDomain: "CertificatesPerDomain", + CertificatesPerDomainPerAccount: "CertificatesPerDomainPerAccount", + CertificatesPerFQDNSet: "CertificatesPerFQDNSet", + FailedAuthorizationsForPausingPerDomainPerAccount: "FailedAuthorizationsForPausingPerDomainPerAccount", } // isValid returns true if the Name is a valid rate limit name. @@ -231,6 +242,15 @@ func validateIdForName(name Name, id string) error { // 'enum:fqdnSet' return validateFQDNSet(id) + case FailedAuthorizationsForPausingPerDomainPerAccount: + if strings.Contains(id, ":") { + // 'enum:regId:domain' for transaction + return validateRegIdDomain(id) + } else { + // 'enum:regId' for overrides + return validateRegId(id) + } + case Unknown: fallthrough diff --git a/ratelimits/names_test.go b/ratelimits/names_test.go index c4d5812d8f4..89eca6baa56 100644 --- a/ratelimits/names_test.go +++ b/ratelimits/names_test.go @@ -134,6 +134,34 @@ func TestValidateIdForName(t *testing.T) { id: "12ea5", err: "invalid regId", }, + { + limit: FailedAuthorizationsForPausingPerDomainPerAccount, + desc: "transaction: valid regId and domain", + id: "12345:example.com", + }, + { + limit: FailedAuthorizationsForPausingPerDomainPerAccount, + desc: "transaction: invalid regId", + id: "12ea5:example.com", + err: "invalid regId", + }, + { + limit: FailedAuthorizationsForPausingPerDomainPerAccount, + desc: "transaction: invalid domain", + id: "12345:examplecom", + err: "name needs at least one dot", + }, + { + limit: FailedAuthorizationsForPausingPerDomainPerAccount, + desc: "override: valid regId", + id: "12345", + }, + { + limit: FailedAuthorizationsForPausingPerDomainPerAccount, + desc: "override: invalid regId", + id: "12ea5", + err: "invalid regId", + }, { limit: CertificatesPerDomainPerAccount, desc: "transaction: valid regId and domain", diff --git a/ratelimits/testdata/working_override_13371338.yml b/ratelimits/testdata/working_override_13371338.yml index a260fedb5f5..97327e510d6 100644 --- a/ratelimits/testdata/working_override_13371338.yml +++ b/ratelimits/testdata/working_override_13371338.yml @@ -12,3 +12,10 @@ ids: - id: 13371338 comment: Used to test the TransactionBuilder +- FailedAuthorizationsForPausingPerDomainPerAccount: + burst: 1337 + count: 1 + period: 24h + ids: + - id: 13371338 + comment: Used to test the TransactionBuilder diff --git a/ratelimits/testdata/working_overrides.yml b/ratelimits/testdata/working_overrides.yml index 584676e87da..f73a5c1893b 100644 --- a/ratelimits/testdata/working_overrides.yml +++ b/ratelimits/testdata/working_overrides.yml @@ -22,3 +22,12 @@ - id: 5678 comment: Foo +- FailedAuthorizationsForPausingPerDomainPerAccount: + burst: 60 + count: 60 + period: 3s + ids: + - id: 1234 + comment: Foo + - id: 5678 + comment: Foo diff --git a/ratelimits/transaction.go b/ratelimits/transaction.go index 4a8dd4435cd..cb5eb489fa5 100644 --- a/ratelimits/transaction.go +++ b/ratelimits/transaction.go @@ -64,9 +64,10 @@ func newDomainBucketKey(name Name, orderName string) (string, error) { return joinWithColon(name.EnumString(), orderName), nil } -// newRegIdDomainBucketKey validates and returns a bucketKey for limits that use -// the 'enum:regId:domain' bucket key format. -func newRegIdDomainBucketKey(name Name, regId int64, orderName string) (string, error) { +// NewRegIdDomainBucketKey validates and returns a bucketKey for limits that use +// the 'enum:regId:domain' bucket key format. This function is exported for use +// in ra.resetAccountPausingLimit. +func NewRegIdDomainBucketKey(name Name, regId int64, orderName string) (string, error) { regIdStr := strconv.FormatInt(regId, 10) err := validateIdForName(name, joinWithColon(regIdStr, orderName)) if err != nil { @@ -257,7 +258,7 @@ func (builder *TransactionBuilder) FailedAuthorizationsPerDomainPerAccountCheckO for _, name := range orderDomains { // FailedAuthorizationsPerDomainPerAccount limit uses the // 'enum:regId:domain' bucket key format for transactions. - perDomainPerAccountBucketKey, err := newRegIdDomainBucketKey(FailedAuthorizationsPerDomainPerAccount, regId, name) + perDomainPerAccountBucketKey, err := NewRegIdDomainBucketKey(FailedAuthorizationsPerDomainPerAccount, regId, name) if err != nil { return nil, err } @@ -292,7 +293,7 @@ func (builder *TransactionBuilder) FailedAuthorizationsPerDomainPerAccountSpendO // FailedAuthorizationsPerDomainPerAccount limit uses the // 'enum:regId:domain' bucket key format for transactions. - perDomainPerAccountBucketKey, err := newRegIdDomainBucketKey(FailedAuthorizationsPerDomainPerAccount, regId, orderDomain) + perDomainPerAccountBucketKey, err := NewRegIdDomainBucketKey(FailedAuthorizationsPerDomainPerAccount, regId, orderDomain) if err != nil { return Transaction{}, err } @@ -304,6 +305,37 @@ func (builder *TransactionBuilder) FailedAuthorizationsPerDomainPerAccountSpendO return txn, nil } +// FailedAuthorizationsForPausingPerDomainPerAccountTransaction returns a +// Transaction for the provided order domain name. An error is returned if +// the order domain name is invalid. This method should be used for spending +// capacity, as a result of a failed authorization. +func (builder *TransactionBuilder) FailedAuthorizationsForPausingPerDomainPerAccountTransaction(regId int64, orderDomain string) (Transaction, error) { + // FailedAuthorizationsForPausingPerDomainPerAccount limit uses the 'enum:regId' + // bucket key format for overrides. + perAccountBucketKey, err := newRegIdBucketKey(FailedAuthorizationsForPausingPerDomainPerAccount, regId) + if err != nil { + return Transaction{}, err + } + limit, err := builder.getLimit(FailedAuthorizationsForPausingPerDomainPerAccount, perAccountBucketKey) + if err != nil && !errors.Is(err, errLimitDisabled) { + return Transaction{}, err + } + + // FailedAuthorizationsForPausingPerDomainPerAccount limit uses the + // 'enum:regId:domain' bucket key format for transactions. + perDomainPerAccountBucketKey, err := NewRegIdDomainBucketKey(FailedAuthorizationsForPausingPerDomainPerAccount, regId, orderDomain) + if err != nil { + return Transaction{}, err + } + + txn, err := newTransaction(limit, perDomainPerAccountBucketKey, 1) + if err != nil { + return Transaction{}, err + } + + return txn, nil +} + // certificatesPerDomainCheckOnlyTransactions returns a slice of Transactions // for the provided order domain names. An error is returned if any of the order // domain names are invalid. This method should be used for checking capacity, @@ -333,7 +365,7 @@ func (builder *TransactionBuilder) certificatesPerDomainCheckOnlyTransactions(re if perAccountLimit.isOverride() { // An override is configured for the CertificatesPerDomainPerAccount // limit. - perAccountPerDomainKey, err := newRegIdDomainBucketKey(CertificatesPerDomainPerAccount, regId, name) + perAccountPerDomainKey, err := NewRegIdDomainBucketKey(CertificatesPerDomainPerAccount, regId, name) if err != nil { return nil, err } @@ -399,7 +431,7 @@ func (builder *TransactionBuilder) CertificatesPerDomainSpendOnlyTransactions(re if perAccountLimit.isOverride() { // An override is configured for the CertificatesPerDomainPerAccount // limit. - perAccountPerDomainKey, err := newRegIdDomainBucketKey(CertificatesPerDomainPerAccount, regId, name) + perAccountPerDomainKey, err := NewRegIdDomainBucketKey(CertificatesPerDomainPerAccount, regId, name) if err != nil { return nil, err } diff --git a/ratelimits/transaction_test.go b/ratelimits/transaction_test.go index 747d14210bf..f8003e2f404 100644 --- a/ratelimits/transaction_test.go +++ b/ratelimits/transaction_test.go @@ -102,6 +102,20 @@ func TestFailedAuthorizationsPerDomainPerAccountTransactions(t *testing.T) { test.Assert(t, txn.limit.isOverride(), "should be an override") } +func TestFailedAuthorizationsForPausingPerDomainPerAccountTransactions(t *testing.T) { + t.Parallel() + + tb, err := NewTransactionBuilder("../test/config-next/wfe2-ratelimit-defaults.yml", "testdata/working_override_13371338.yml") + test.AssertNotError(t, err, "creating TransactionBuilder") + + // A transaction for the per-account limit override. + txn, err := tb.FailedAuthorizationsForPausingPerDomainPerAccountTransaction(13371338, "so.many.labels.here.example.com") + test.AssertNotError(t, err, "creating transaction") + test.AssertEquals(t, txn.bucketKey, "8:13371338:so.many.labels.here.example.com") + test.Assert(t, txn.check && txn.spend, "should be check and spend") + test.Assert(t, txn.limit.isOverride(), "should be an override") +} + func TestCertificatesPerDomainTransactions(t *testing.T) { t.Parallel() diff --git a/test/config-next/ra.json b/test/config-next/ra.json index 2e1a313a463..6e84c2e7ba6 100644 --- a/test/config-next/ra.json +++ b/test/config-next/ra.json @@ -128,6 +128,7 @@ "features": { "AsyncFinalize": true, "UseKvLimitsForNewOrder": true, + "AutomaticallyPauseZombieClients": true, "IncrementRateLimits": true }, "ctLogs": { diff --git a/test/config-next/wfe2-ratelimit-defaults.yml b/test/config-next/wfe2-ratelimit-defaults.yml index c24e854c2b3..d934b508cc8 100644 --- a/test/config-next/wfe2-ratelimit-defaults.yml +++ b/test/config-next/wfe2-ratelimit-defaults.yml @@ -14,6 +14,18 @@ FailedAuthorizationsPerDomainPerAccount: count: 3 burst: 3 period: 5m +# The burst represents failing 40 times per day for 90 days. The count and +# period grant one "freebie" failure per day. In combination, these parameters +# mean that: +# - Failing 120 times per day results in being paused after 30.25 days +# - Failing 40 times per day results in being paused after 92.3 days +# - Failing 20 times per day results in being paused after 6.2 months +# - Failing 4 times per day results in being paused after 3.3 years +# - Failing once per day results in never being paused +FailedAuthorizationsForPausingPerDomainPerAccount: + count: 1 + burst: 3600 + period: 24h NewOrdersPerAccount: count: 1500 burst: 1500