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:
Jacob Hoffman-Andrews 2021-12-06 14:46:46 -08:00 committed by GitHub
parent cbd24db64b
commit 3d7206a183
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 302 additions and 108 deletions

View File

@ -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

View File

@ -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)
}

View File

@ -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 {

View File

@ -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())
}

View File

@ -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)
}

View File

@ -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

View File

@ -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()

View File

@ -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{

View File

@ -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)
}

View File

@ -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())
}

View File

@ -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,