ocsp-updater: add support for writing to Redis (#5825)
If configured, ocsp-updater will write responses to Redis in parallel with MariaDB, giving up if Redis is slower and incrementing a stat. Factors out the ShortIDIssuer concept from rocsp-tool into rocsp_config.
This commit is contained in:
parent
cbd24db64b
commit
3d7206a183
|
|
@ -16,6 +16,8 @@ import (
|
|||
bgrpc "github.com/letsencrypt/boulder/grpc"
|
||||
"github.com/letsencrypt/boulder/ocsp_updater"
|
||||
ocsp_updater_config "github.com/letsencrypt/boulder/ocsp_updater/config"
|
||||
"github.com/letsencrypt/boulder/rocsp"
|
||||
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
|
||||
"github.com/letsencrypt/boulder/sa"
|
||||
)
|
||||
|
||||
|
|
@ -112,6 +114,15 @@ func main() {
|
|||
|
||||
clk := cmd.Clock()
|
||||
|
||||
redisConf := c.OCSPUpdater.Redis
|
||||
var rocspClient *rocsp.WritingClient
|
||||
if redisConf != nil {
|
||||
rocspClient, err = rocsp_config.MakeClient(redisConf, clk)
|
||||
cmd.FailOnError(err, "making Redis client")
|
||||
}
|
||||
issuers, err := rocsp_config.LoadIssuers(c.OCSPUpdater.Issuers)
|
||||
cmd.FailOnError(err, "loading issuers")
|
||||
|
||||
tlsConfig, err := c.OCSPUpdater.TLS.Load()
|
||||
cmd.FailOnError(err, "TLS config")
|
||||
clientMetrics := bgrpc.NewClientMetrics(stats)
|
||||
|
|
@ -129,6 +140,8 @@ func main() {
|
|||
clk,
|
||||
db,
|
||||
readOnlyDb,
|
||||
rocspClient,
|
||||
issuers,
|
||||
serialSuffixes,
|
||||
ogc,
|
||||
// Necessary evil for now
|
||||
|
|
|
|||
|
|
@ -15,13 +15,14 @@ import (
|
|||
capb "github.com/letsencrypt/boulder/ca/proto"
|
||||
"github.com/letsencrypt/boulder/core"
|
||||
"github.com/letsencrypt/boulder/rocsp"
|
||||
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
|
||||
"github.com/letsencrypt/boulder/sa"
|
||||
"github.com/letsencrypt/boulder/test/ocsp/helper"
|
||||
"golang.org/x/crypto/ocsp"
|
||||
)
|
||||
|
||||
type client struct {
|
||||
issuers []shortIDIssuer
|
||||
issuers []rocsp_config.ShortIDIssuer
|
||||
redis *rocsp.WritingClient
|
||||
db *sql.DB // optional
|
||||
ocspGenerator capb.OCSPGeneratorClient
|
||||
|
|
@ -218,13 +219,13 @@ func (cl *client) signAndStoreResponses(ctx context.Context, input <-chan *sa.Ce
|
|||
}
|
||||
// ttl is the lifetime of the certificate
|
||||
ttl := cl.clk.Now().Sub(status.NotAfter)
|
||||
issuer, err := findIssuerByID(status.IssuerID, cl.issuers)
|
||||
issuer, err := rocsp_config.FindIssuerByID(status.IssuerID, cl.issuers)
|
||||
if err != nil {
|
||||
output <- processResult{id: uint64(status.ID), err: err}
|
||||
continue
|
||||
}
|
||||
|
||||
err = cl.redis.StoreResponse(ctx, result.Response, issuer.shortID, ttl)
|
||||
err = cl.redis.StoreResponse(ctx, result.Response, issuer.ShortID(), ttl)
|
||||
if err != nil {
|
||||
output <- processResult{id: uint64(status.ID), err: err}
|
||||
} else {
|
||||
|
|
@ -261,7 +262,7 @@ func (cl *client) storeResponse(ctx context.Context, respBytes []byte, ttl *time
|
|||
if err != nil {
|
||||
return fmt.Errorf("parsing response: %w", err)
|
||||
}
|
||||
issuer, err := findIssuerByName(resp, cl.issuers)
|
||||
issuer, err := rocsp_config.FindIssuerByName(resp, cl.issuers)
|
||||
if err != nil {
|
||||
return fmt.Errorf("finding issuer for response: %w", err)
|
||||
}
|
||||
|
|
@ -297,7 +298,7 @@ func (cl *client) storeResponse(ctx context.Context, respBytes []byte, ttl *time
|
|||
ttl.Hours(),
|
||||
)
|
||||
|
||||
err = cl.redis.StoreResponse(ctx, respBytes, issuer.shortID, *ttl)
|
||||
err = cl.redis.StoreResponse(ctx, respBytes, issuer.ShortID(), *ttl)
|
||||
if err != nil {
|
||||
return fmt.Errorf("storing response: %w", err)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,6 +12,7 @@ import (
|
|||
"github.com/letsencrypt/boulder/core"
|
||||
"github.com/letsencrypt/boulder/log"
|
||||
"github.com/letsencrypt/boulder/rocsp"
|
||||
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
|
||||
"github.com/letsencrypt/boulder/sa"
|
||||
"github.com/letsencrypt/boulder/test"
|
||||
"github.com/letsencrypt/boulder/test/vars"
|
||||
|
|
@ -90,7 +91,7 @@ func TestStoreResponse(t *testing.T) {
|
|||
}, issuerKey)
|
||||
test.AssertNotError(t, err, "creating OCSP response")
|
||||
|
||||
issuers, err := loadIssuers(map[string]int{
|
||||
issuers, err := rocsp_config.LoadIssuers(map[string]int{
|
||||
"../../test/hierarchy/int-e1.cert.pem": 23,
|
||||
})
|
||||
if err != nil {
|
||||
|
|
|
|||
|
|
@ -1,80 +0,0 @@
|
|||
package notmain
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"crypto/x509/pkix"
|
||||
"encoding/asn1"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/letsencrypt/boulder/issuance"
|
||||
"golang.org/x/crypto/ocsp"
|
||||
)
|
||||
|
||||
type shortIDIssuer struct {
|
||||
*issuance.Certificate
|
||||
subject pkix.RDNSequence
|
||||
shortID byte
|
||||
issuerID issuance.IssuerID
|
||||
issuerNameID issuance.IssuerNameID
|
||||
}
|
||||
|
||||
func loadIssuers(input map[string]int) ([]shortIDIssuer, error) {
|
||||
var issuers []shortIDIssuer
|
||||
for issuerFile, shortID := range input {
|
||||
if shortID > 255 || shortID < 0 {
|
||||
return nil, fmt.Errorf("invalid shortID %d (must be byte)", shortID)
|
||||
}
|
||||
cert, err := issuance.LoadCertificate(issuerFile)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("reading issuer: %w", err)
|
||||
}
|
||||
var subject pkix.RDNSequence
|
||||
_, err = asn1.Unmarshal(cert.Certificate.RawSubject, &subject)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("parsing issuer.RawSubject: %w", err)
|
||||
}
|
||||
var shortID byte = byte(shortID)
|
||||
for _, issuer := range issuers {
|
||||
if issuer.shortID == shortID {
|
||||
return nil, fmt.Errorf("duplicate shortID in config file: %d (for %q and %q)", shortID, issuer.subject, subject)
|
||||
}
|
||||
if !issuer.IsCA {
|
||||
return nil, fmt.Errorf("certificate for %q is not a CA certificate", subject)
|
||||
}
|
||||
}
|
||||
issuers = append(issuers, shortIDIssuer{
|
||||
Certificate: cert,
|
||||
subject: subject,
|
||||
shortID: shortID,
|
||||
issuerID: cert.ID(),
|
||||
issuerNameID: cert.NameID(),
|
||||
})
|
||||
}
|
||||
return issuers, nil
|
||||
}
|
||||
|
||||
func findIssuerByID(longID int64, issuers []shortIDIssuer) (*shortIDIssuer, error) {
|
||||
for _, iss := range issuers {
|
||||
if iss.issuerNameID == issuance.IssuerNameID(longID) || iss.issuerID == issuance.IssuerID(longID) {
|
||||
return &iss, nil
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("no issuer found for an ID in certificateStatus: %d", longID)
|
||||
}
|
||||
|
||||
func findIssuerByName(resp *ocsp.Response, issuers []shortIDIssuer) (*shortIDIssuer, error) {
|
||||
var responder pkix.RDNSequence
|
||||
_, err := asn1.Unmarshal(resp.RawResponderName, &responder)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("parsing resp.RawResponderName: %w", err)
|
||||
}
|
||||
var responders strings.Builder
|
||||
for _, issuer := range issuers {
|
||||
fmt.Fprintf(&responders, "%s\n", issuer.subject)
|
||||
if bytes.Equal(issuer.RawSubject, resp.RawResponderName) {
|
||||
return &issuer, nil
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("no issuer found matching OCSP response for %s. Available issuers:\n%s\n", responder, responders.String())
|
||||
}
|
||||
|
|
@ -90,7 +90,7 @@ func main2() error {
|
|||
return fmt.Errorf("reading JSON config file: %w", err)
|
||||
}
|
||||
|
||||
issuers, err := loadIssuers(c.ROCSPTool.Issuers)
|
||||
issuers, err := rocsp_config.LoadIssuers(c.ROCSPTool.Issuers)
|
||||
if err != nil {
|
||||
return fmt.Errorf("loading issuers: %w", err)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,9 @@
|
|||
package ocsp_updater_config
|
||||
|
||||
import "github.com/letsencrypt/boulder/cmd"
|
||||
import (
|
||||
"github.com/letsencrypt/boulder/cmd"
|
||||
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
|
||||
)
|
||||
|
||||
// Config provides the various window tick times and batch sizes needed
|
||||
// for the OCSP updater
|
||||
|
|
@ -8,6 +11,14 @@ type Config struct {
|
|||
cmd.ServiceConfig
|
||||
DB cmd.DBConfig
|
||||
ReadOnlyDB cmd.DBConfig
|
||||
Redis *rocsp_config.RedisConfig
|
||||
|
||||
// Issuers is a map from filenames to short issuer IDs.
|
||||
// Each filename must contain an issuer certificate. The short issuer
|
||||
// IDs are arbitrarily assigned and must be consistent across OCSP
|
||||
// components. For production we'll use the number part of the CN, i.e.
|
||||
// E1 -> 1, R3 -> 3, etc.
|
||||
Issuers map[string]int
|
||||
|
||||
OldOCSPWindow cmd.ConfigDuration
|
||||
OldOCSPBatchSize int
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import (
|
|||
"github.com/letsencrypt/boulder/core"
|
||||
blog "github.com/letsencrypt/boulder/log"
|
||||
ocsp_updater_config "github.com/letsencrypt/boulder/ocsp_updater/config"
|
||||
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
|
||||
"github.com/letsencrypt/boulder/sa"
|
||||
)
|
||||
|
||||
|
|
@ -34,6 +35,10 @@ type ocspDb interface {
|
|||
Exec(query string, args ...interface{}) (sql.Result, error)
|
||||
}
|
||||
|
||||
type rocspClientInterface interface {
|
||||
StoreResponse(ctx context.Context, respBytes []byte, shortIssuerID byte, ttl time.Duration) error
|
||||
}
|
||||
|
||||
// failCounter provides a concurrent safe counter.
|
||||
type failCounter struct {
|
||||
mu sync.Mutex
|
||||
|
|
@ -63,8 +68,11 @@ type OCSPUpdater struct {
|
|||
log blog.Logger
|
||||
clk clock.Clock
|
||||
|
||||
db ocspDb
|
||||
readOnlyDb ocspReadOnlyDb
|
||||
db ocspDb
|
||||
readOnlyDb ocspReadOnlyDb
|
||||
rocspClient rocspClientInterface
|
||||
|
||||
issuers []rocsp_config.ShortIDIssuer
|
||||
|
||||
ogc capb.OCSPGeneratorClient
|
||||
|
||||
|
|
@ -89,6 +97,7 @@ type OCSPUpdater struct {
|
|||
genStoreHistogram prometheus.Histogram
|
||||
generatedCounter *prometheus.CounterVec
|
||||
storedCounter *prometheus.CounterVec
|
||||
storedRedisCounter *prometheus.CounterVec
|
||||
markExpiredCounter *prometheus.CounterVec
|
||||
findStaleOCSPCounter *prometheus.CounterVec
|
||||
}
|
||||
|
|
@ -98,6 +107,8 @@ func New(
|
|||
clk clock.Clock,
|
||||
db ocspDb,
|
||||
readOnlyDb ocspReadOnlyDb,
|
||||
rocspClient rocspClientInterface,
|
||||
issuers []rocsp_config.ShortIDIssuer,
|
||||
serialSuffixes []string,
|
||||
ogc capb.OCSPGeneratorClient,
|
||||
config ocsp_updater_config.Config,
|
||||
|
|
@ -142,6 +153,10 @@ func New(
|
|||
Help: "A counter of OCSP response generation calls labeled by result",
|
||||
}, []string{"result"})
|
||||
stats.MustRegister(generatedCounter)
|
||||
storedRedisCounter := prometheus.NewCounterVec(prometheus.CounterOpts{
|
||||
Name: "ocsp_updater_stored_redis",
|
||||
Help: "A counter of OCSP response storage calls labeled by result",
|
||||
}, []string{"result"})
|
||||
storedCounter := prometheus.NewCounterVec(prometheus.CounterOpts{
|
||||
Name: "ocsp_updater_stored",
|
||||
Help: "A counter of OCSP response storage calls labeled by result",
|
||||
|
|
@ -170,10 +185,16 @@ func New(
|
|||
}, []string{"result"})
|
||||
stats.MustRegister(findStaleOCSPCounter)
|
||||
|
||||
var rocspClientInterface rocspClientInterface
|
||||
if rocspClient != nil {
|
||||
rocspClientInterface = rocspClient
|
||||
}
|
||||
updater := OCSPUpdater{
|
||||
clk: clk,
|
||||
db: db,
|
||||
readOnlyDb: readOnlyDb,
|
||||
rocspClient: rocspClientInterface,
|
||||
issuers: issuers,
|
||||
ogc: ogc,
|
||||
log: log,
|
||||
ocspMinTimeToExpiry: config.OCSPMinTimeToExpiry.Duration,
|
||||
|
|
@ -181,6 +202,7 @@ func New(
|
|||
genStoreHistogram: genStoreHistogram,
|
||||
generatedCounter: generatedCounter,
|
||||
storedCounter: storedCounter,
|
||||
storedRedisCounter: storedRedisCounter,
|
||||
markExpiredCounter: markExpiredCounter,
|
||||
findStaleOCSPCounter: findStaleOCSPCounter,
|
||||
stalenessHistogram: stalenessHistogram,
|
||||
|
|
@ -297,7 +319,30 @@ func (updater *OCSPUpdater) generateResponse(ctx context.Context, status sa.Cert
|
|||
}
|
||||
|
||||
// storeResponse stores a given CertificateStatus in the database.
|
||||
func (updater *OCSPUpdater) storeResponse(status *sa.CertStatusMetadata) error {
|
||||
func (updater *OCSPUpdater) storeResponse(ctx context.Context, status *sa.CertStatusMetadata) error {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
|
||||
if updater.rocspClient != nil {
|
||||
go func() {
|
||||
ttl := status.NotAfter.Sub(updater.clk.Now())
|
||||
shortIssuerID, err := rocsp_config.FindIssuerByID(status.IssuerID, updater.issuers)
|
||||
if err != nil {
|
||||
updater.storedRedisCounter.WithLabelValues("missing issuer").Inc()
|
||||
return
|
||||
}
|
||||
err = updater.rocspClient.StoreResponse(ctx, status.OCSPResponse, shortIssuerID.ShortID(), ttl)
|
||||
if err != nil {
|
||||
if errors.Is(err, context.Canceled) {
|
||||
updater.storedRedisCounter.WithLabelValues("canceled").Inc()
|
||||
} else {
|
||||
updater.storedRedisCounter.WithLabelValues("failed").Inc()
|
||||
}
|
||||
} else {
|
||||
updater.storedRedisCounter.WithLabelValues("success").Inc()
|
||||
}
|
||||
}()
|
||||
}
|
||||
// Update the certificateStatus table with the new OCSP response, the status
|
||||
// WHERE is used make sure we don't overwrite a revoked response with a one
|
||||
// containing a 'good' status.
|
||||
|
|
@ -311,6 +356,12 @@ func (updater *OCSPUpdater) storeResponse(status *sa.CertStatusMetadata) error {
|
|||
status.ID,
|
||||
string(status.Status),
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
updater.storedCounter.WithLabelValues("failed").Inc()
|
||||
} else {
|
||||
updater.storedCounter.WithLabelValues("success").Inc()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
@ -385,7 +436,7 @@ func (updater *OCSPUpdater) generateOCSPResponses(ctx context.Context, staleStat
|
|||
}
|
||||
updater.generatedCounter.WithLabelValues("success").Inc()
|
||||
|
||||
err = updater.storeResponse(meta)
|
||||
err = updater.storeResponse(ctx, meta)
|
||||
if err != nil {
|
||||
updater.log.AuditErrf("Failed to store OCSP response: %s", err)
|
||||
updater.storedCounter.WithLabelValues("failed").Inc()
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@ import (
|
|||
blog "github.com/letsencrypt/boulder/log"
|
||||
"github.com/letsencrypt/boulder/metrics"
|
||||
ocsp_updater_config "github.com/letsencrypt/boulder/ocsp_updater/config"
|
||||
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
|
||||
"github.com/letsencrypt/boulder/sa"
|
||||
sapb "github.com/letsencrypt/boulder/sa/proto"
|
||||
"github.com/letsencrypt/boulder/sa/satest"
|
||||
|
|
@ -43,6 +44,13 @@ func (ca *mockOCSP) GenerateOCSP(_ context.Context, req *capb.GenerateOCSPReques
|
|||
return &capb.OCSPResponse{Response: []byte{1, 2, 3}}, nil
|
||||
}
|
||||
|
||||
type noopROCSP struct {
|
||||
}
|
||||
|
||||
func (_ noopROCSP) StoreResponse(_ context.Context, _ []byte, _ byte, _ time.Duration) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
var log = blog.UseMock()
|
||||
|
||||
func setup(t *testing.T) (*OCSPUpdater, sapb.StorageAuthorityClient, *db.WrappedMap, clock.FakeClock, func()) {
|
||||
|
|
@ -64,6 +72,8 @@ func setup(t *testing.T) (*OCSPUpdater, sapb.StorageAuthorityClient, *db.Wrapped
|
|||
fc,
|
||||
dbMap,
|
||||
readOnlyDb,
|
||||
noopROCSP{},
|
||||
nil,
|
||||
strings.Fields("0 1 2 3 4 5 6 7 8 9 a b c d e f"),
|
||||
&mockOCSP{},
|
||||
ocsp_updater_config.Config{
|
||||
|
|
@ -153,10 +163,91 @@ func TestGenerateAndStoreOCSPResponse(t *testing.T) {
|
|||
|
||||
meta, err := updater.generateResponse(ctx, status)
|
||||
test.AssertNotError(t, err, "Couldn't generate OCSP response")
|
||||
err = updater.storeResponse(meta)
|
||||
err = updater.storeResponse(context.Background(), meta)
|
||||
test.AssertNotError(t, err, "Couldn't store certificate status")
|
||||
}
|
||||
|
||||
type rocspStorage struct {
|
||||
shortIDIssuer byte
|
||||
response []byte
|
||||
ttl time.Duration
|
||||
}
|
||||
|
||||
type recordingROCSP struct {
|
||||
sync.Mutex
|
||||
storage []rocspStorage
|
||||
}
|
||||
|
||||
func (rr *recordingROCSP) get() []rocspStorage {
|
||||
rr.Lock()
|
||||
defer rr.Unlock()
|
||||
var ret []rocspStorage
|
||||
return append(ret, rr.storage...)
|
||||
}
|
||||
|
||||
func (rr *recordingROCSP) StoreResponse(ctx context.Context, respBytes []byte, shortIssuerID byte, ttl time.Duration) error {
|
||||
rr.Lock()
|
||||
defer rr.Unlock()
|
||||
rr.storage = append(rr.storage, rocspStorage{
|
||||
shortIDIssuer: shortIssuerID,
|
||||
response: respBytes,
|
||||
ttl: ttl,
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
// A mock ocspDb that sleeps for 50ms when Exec is called.
|
||||
type mockDBBlocksOnExec struct{}
|
||||
|
||||
func (mdboe *mockDBBlocksOnExec) Query(query string, args ...interface{}) (*sql.Rows, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (mdboe *mockDBBlocksOnExec) Exec(query string, args ...interface{}) (sql.Result, error) {
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func TestROCSP(t *testing.T) {
|
||||
updater, sac, _, fc, cleanUp := setup(t)
|
||||
defer cleanUp()
|
||||
|
||||
reg := satest.CreateWorkingRegistration(t, sac)
|
||||
parsedCert, err := core.LoadCert("testdata/test-cert.pem")
|
||||
test.AssertNotError(t, err, "Couldn't read test certificate")
|
||||
_, err = sac.AddPrecertificate(ctx, &sapb.AddCertificateRequest{
|
||||
Der: parsedCert.Raw,
|
||||
RegID: reg.Id,
|
||||
Ocsp: nil,
|
||||
Issued: nowNano(fc),
|
||||
IssuerID: 66283756913588288,
|
||||
})
|
||||
test.AssertNotError(t, err, "Couldn't add test-cert.pem")
|
||||
|
||||
recorder := &recordingROCSP{}
|
||||
updater.rocspClient = recorder
|
||||
updater.issuers, err = rocsp_config.LoadIssuers(
|
||||
map[string]int{
|
||||
"../test/hierarchy/int-e1.cert.pem": 23,
|
||||
},
|
||||
)
|
||||
test.AssertNotError(t, err, "loading issuers")
|
||||
updater.db = &mockDBBlocksOnExec{}
|
||||
|
||||
err = updater.storeResponse(context.Background(), &sa.CertStatusMetadata{
|
||||
CertificateStatus: core.CertificateStatus{
|
||||
OCSPResponse: []byte("fake response"),
|
||||
Serial: "fake serial",
|
||||
IssuerID: 66283756913588288,
|
||||
},
|
||||
})
|
||||
test.AssertNotError(t, err, "Couldn't store certificate status")
|
||||
storage := recorder.get()
|
||||
test.AssertEquals(t, len(storage), 1)
|
||||
|
||||
test.AssertByteEquals(t, storage[0].response, []byte("fake response"))
|
||||
}
|
||||
|
||||
// findStaleOCSPResponsesBuffered runs findStaleOCSPResponses and returns
|
||||
// it as a buffered channel. This is helpful for tests that want to test
|
||||
// the length of the channel.
|
||||
|
|
@ -270,7 +361,7 @@ func TestFindStaleOCSPResponses(t *testing.T) {
|
|||
// ocspLastUpdate field for this cert.
|
||||
meta, err := updater.generateResponse(ctx, status)
|
||||
test.AssertNotError(t, err, "Couldn't generate OCSP response")
|
||||
err = updater.storeResponse(meta)
|
||||
err = updater.storeResponse(context.Background(), meta)
|
||||
test.AssertNotError(t, err, "Couldn't store OCSP response")
|
||||
|
||||
// We should have 0 stale responses now.
|
||||
|
|
@ -438,7 +529,7 @@ func TestStoreResponseGuard(t *testing.T) {
|
|||
// Attempt to update OCSP response where status.Status is good but stored status
|
||||
// is revoked, this should fail silently
|
||||
status.OCSPResponse = []byte("newfakeocspbytes")
|
||||
err = updater.storeResponse(&status)
|
||||
err = updater.storeResponse(context.Background(), &status)
|
||||
test.AssertNotError(t, err, "Failed to update certificate status")
|
||||
|
||||
// Make sure the OCSP response hasn't actually changed
|
||||
|
|
@ -448,7 +539,7 @@ func TestStoreResponseGuard(t *testing.T) {
|
|||
|
||||
// Changing the status to the stored status should allow the update to occur
|
||||
status.Status = core.OCSPStatusRevoked
|
||||
err = updater.storeResponse(&status)
|
||||
err = updater.storeResponse(context.Background(), &status)
|
||||
test.AssertNotError(t, err, "Failed to updated certificate status")
|
||||
|
||||
// Make sure the OCSP response has been updated
|
||||
|
|
@ -622,6 +713,8 @@ func mkNewUpdaterWithStrings(t *testing.T, shards []string) (*OCSPUpdater, error
|
|||
fc,
|
||||
dbMap,
|
||||
dbMap,
|
||||
noopROCSP{},
|
||||
nil,
|
||||
shards,
|
||||
&mockOCSP{},
|
||||
ocsp_updater_config.Config{
|
||||
|
|
|
|||
|
|
@ -1,4 +1,4 @@
|
|||
package notmain
|
||||
package rocsp_config
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
|
|
@ -14,13 +14,13 @@ func TestLoadIssuers(t *testing.T) {
|
|||
"../../test/hierarchy/int-e1.cert.pem": 23,
|
||||
"../../test/hierarchy/int-r3.cert.pem": 99,
|
||||
}
|
||||
output, err := loadIssuers(input)
|
||||
output, err := LoadIssuers(input)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
var e1 *shortIDIssuer
|
||||
var r3 *shortIDIssuer
|
||||
var e1 *ShortIDIssuer
|
||||
var r3 *ShortIDIssuer
|
||||
|
||||
for i, v := range output {
|
||||
if strings.Contains(v.Certificate.Subject.String(), "E1") {
|
||||
|
|
@ -42,7 +42,7 @@ func TestFindIssuerByName(t *testing.T) {
|
|||
"../../test/hierarchy/int-e1.cert.pem": 23,
|
||||
"../../test/hierarchy/int-r3.cert.pem": 99,
|
||||
}
|
||||
issuers, err := loadIssuers(input)
|
||||
issuers, err := LoadIssuers(input)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
|
@ -60,7 +60,7 @@ func TestFindIssuerByName(t *testing.T) {
|
|||
RawResponderName: elephant,
|
||||
}
|
||||
|
||||
issuer, err := findIssuerByName(ocspResp, issuers)
|
||||
issuer, err := FindIssuerByName(ocspResp, issuers)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't find issuer: %s", err)
|
||||
}
|
||||
|
|
@ -71,7 +71,7 @@ func TestFindIssuerByName(t *testing.T) {
|
|||
RawResponderName: rhino,
|
||||
}
|
||||
|
||||
issuer, err = findIssuerByName(ocspResp, issuers)
|
||||
issuer, err = FindIssuerByName(ocspResp, issuers)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't find issuer: %s", err)
|
||||
}
|
||||
|
|
@ -84,34 +84,34 @@ func TestFindIssuerByID(t *testing.T) {
|
|||
"../../test/hierarchy/int-e1.cert.pem": 23,
|
||||
"../../test/hierarchy/int-r3.cert.pem": 99,
|
||||
}
|
||||
issuers, err := loadIssuers(input)
|
||||
issuers, err := LoadIssuers(input)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// an IssuerNameID
|
||||
issuer, err := findIssuerByID(66283756913588288, issuers)
|
||||
issuer, err := FindIssuerByID(66283756913588288, issuers)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't find issuer: %s", err)
|
||||
}
|
||||
test.AssertEquals(t, issuer.shortID, uint8(23))
|
||||
|
||||
// an IssuerID
|
||||
issuer, err = findIssuerByID(2823400738, issuers)
|
||||
issuer, err = FindIssuerByID(2823400738, issuers)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't find issuer: %s", err)
|
||||
}
|
||||
test.AssertEquals(t, issuer.shortID, uint8(23))
|
||||
|
||||
// an IssuerNameID
|
||||
issuer, err = findIssuerByID(58923463773186183, issuers)
|
||||
issuer, err = FindIssuerByID(58923463773186183, issuers)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't find issuer: %s", err)
|
||||
}
|
||||
test.AssertEquals(t, issuer.shortID, uint8(99))
|
||||
|
||||
// an IssuerID
|
||||
issuer, err = findIssuerByID(2890189813, issuers)
|
||||
issuer, err = FindIssuerByID(2890189813, issuers)
|
||||
if err != nil {
|
||||
t.Fatalf("couldn't find issuer: %s", err)
|
||||
}
|
||||
|
|
@ -1,12 +1,18 @@
|
|||
package rocsp_config
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"crypto/x509/pkix"
|
||||
"encoding/asn1"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/go-redis/redis/v8"
|
||||
"github.com/jmhodges/clock"
|
||||
"github.com/letsencrypt/boulder/cmd"
|
||||
"github.com/letsencrypt/boulder/issuance"
|
||||
"github.com/letsencrypt/boulder/rocsp"
|
||||
"golang.org/x/crypto/ocsp"
|
||||
)
|
||||
|
||||
// RedisConfig contains the configuration needed to act as a Redis client.
|
||||
|
|
@ -68,6 +74,7 @@ type RedisConfig struct {
|
|||
IdleCheckFrequency cmd.ConfigDuration
|
||||
}
|
||||
|
||||
// MakeClient produces a *rocsp.WritingClient from a config.
|
||||
func MakeClient(c *RedisConfig, clk clock.Clock) (*rocsp.WritingClient, error) {
|
||||
password, err := c.PasswordConfig.Pass()
|
||||
if err != nil {
|
||||
|
|
@ -104,6 +111,7 @@ func MakeClient(c *RedisConfig, clk clock.Clock) (*rocsp.WritingClient, error) {
|
|||
return rocsp.NewWritingClient(rdb, timeout, clk), nil
|
||||
}
|
||||
|
||||
// MakeReadClient produces a *rocsp.Client from a config.
|
||||
func MakeReadClient(c *RedisConfig, clk clock.Clock) (*rocsp.Client, error) {
|
||||
password, err := c.PasswordConfig.Pass()
|
||||
if err != nil {
|
||||
|
|
@ -138,3 +146,80 @@ func MakeReadClient(c *RedisConfig, clk clock.Clock) (*rocsp.Client, error) {
|
|||
})
|
||||
return rocsp.NewClient(rdb, timeout, clk), nil
|
||||
}
|
||||
|
||||
// A ShortIDIssuer combines an issuance.Certificate with some fields necessary
|
||||
// to process OCSP responses: the subject name and the shortID.
|
||||
type ShortIDIssuer struct {
|
||||
*issuance.Certificate
|
||||
subject pkix.RDNSequence
|
||||
shortID byte
|
||||
}
|
||||
|
||||
// LoadIssuers takes a map where the keys are filenames and the values are the
|
||||
// corresponding short issuer ID. It loads issuer certificates from the given
|
||||
// files and produces a []ShortIDIssuer.
|
||||
func LoadIssuers(input map[string]int) ([]ShortIDIssuer, error) {
|
||||
var issuers []ShortIDIssuer
|
||||
for issuerFile, shortID := range input {
|
||||
if shortID > 255 || shortID < 0 {
|
||||
return nil, fmt.Errorf("invalid shortID %d (must be byte)", shortID)
|
||||
}
|
||||
cert, err := issuance.LoadCertificate(issuerFile)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("reading issuer: %w", err)
|
||||
}
|
||||
var subject pkix.RDNSequence
|
||||
_, err = asn1.Unmarshal(cert.Certificate.RawSubject, &subject)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("parsing issuer.RawSubject: %w", err)
|
||||
}
|
||||
var shortID byte = byte(shortID)
|
||||
for _, issuer := range issuers {
|
||||
if issuer.shortID == shortID {
|
||||
return nil, fmt.Errorf("duplicate shortID '%d' in (for %q and %q) in config file", shortID, issuer.subject, subject)
|
||||
}
|
||||
if !issuer.IsCA {
|
||||
return nil, fmt.Errorf("certificate for %q is not a CA certificate", subject)
|
||||
}
|
||||
}
|
||||
issuers = append(issuers, ShortIDIssuer{
|
||||
Certificate: cert,
|
||||
subject: subject,
|
||||
shortID: shortID,
|
||||
})
|
||||
}
|
||||
return issuers, nil
|
||||
}
|
||||
|
||||
// ShortID returns the short ID of an issuer. The short ID is a single byte that
|
||||
// is unique for that issuer.
|
||||
func (si *ShortIDIssuer) ShortID() byte {
|
||||
return si.shortID
|
||||
}
|
||||
|
||||
// FindIssuerByID returns the issuer that matches the given IssuerID or IssuerNameID.
|
||||
func FindIssuerByID(longID int64, issuers []ShortIDIssuer) (*ShortIDIssuer, error) {
|
||||
for _, iss := range issuers {
|
||||
if iss.NameID() == issuance.IssuerNameID(longID) || iss.ID() == issuance.IssuerID(longID) {
|
||||
return &iss, nil
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("no issuer found for an ID in certificateStatus: %d", longID)
|
||||
}
|
||||
|
||||
// FindIssuerByID returns the issuer with a Subject matching the *ocsp.Response.
|
||||
func FindIssuerByName(resp *ocsp.Response, issuers []ShortIDIssuer) (*ShortIDIssuer, error) {
|
||||
var responder pkix.RDNSequence
|
||||
_, err := asn1.Unmarshal(resp.RawResponderName, &responder)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("parsing resp.RawResponderName: %w", err)
|
||||
}
|
||||
var responders strings.Builder
|
||||
for _, issuer := range issuers {
|
||||
fmt.Fprintf(&responders, "%s\n", issuer.subject)
|
||||
if bytes.Equal(issuer.RawSubject, resp.RawResponderName) {
|
||||
return &issuer, nil
|
||||
}
|
||||
}
|
||||
return nil, fmt.Errorf("no issuer found matching OCSP response for %s. Available issuers:\n%s\n", responder, responders.String())
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,6 +8,25 @@
|
|||
"dbConnectFile": "test/secrets/ocsp_updater_ro_dburl",
|
||||
"maxOpenConns": 100
|
||||
},
|
||||
"redis": {
|
||||
"username": "ocsp-updater",
|
||||
"passwordFile": "test/secrets/rocsp_tool_password",
|
||||
"addrs": [
|
||||
"10.33.33.7:4218"
|
||||
],
|
||||
"timeout": "5s",
|
||||
"tls": {
|
||||
"caCertFile": "test/redis-tls/minica.pem",
|
||||
"certFile": "test/redis-tls/boulder/cert.pem",
|
||||
"keyFile": "test/redis-tls/boulder/key.pem"
|
||||
}
|
||||
},
|
||||
"issuers": {
|
||||
".hierarchy/intermediate-cert-ecdsa-a.pem": 1,
|
||||
".hierarchy/intermediate-cert-ecdsa-b.pem": 2,
|
||||
".hierarchy/intermediate-cert-rsa-a.pem": 3,
|
||||
".hierarchy/intermediate-cert-rsa-b.pem": 4
|
||||
},
|
||||
"oldOCSPWindow": "2s",
|
||||
"oldOCSPBatchSize": 5000,
|
||||
"parallelGenerateOCSPRequests": 10,
|
||||
|
|
|
|||
Loading…
Reference in New Issue