rocsp: remove Metadata (#6206)

The metadata values were planned to be used for scanning Redis in
ocsp-updater. Since we won't do that, remove it. Happily, this also
allows us to get rid of shortIssuerId.

Removing the issuer check in rocsp_sa.go uncovered a "boxed nil" problem:
SA was doing a nil check against an interface field that in practice was
never nil (because it was promoted from a concrete type at construction
time). So we would always hit the ROCSP path. But one of the first steps
in that path was looking up an issuer ID. Since `test/config` never
had the issuers set, we would look up the issuer ID, not find it, and
return an error before we attempted to call storeResponse. To fix this,
I made `NewSQLStorageAuthority` take a concrete `*rocsp.WritingClient`
instead of an interface, and check for nil before assigning it to an
internal interface field.

Built on top of #6201.
This commit is contained in:
Jacob Hoffman-Andrews 2022-07-05 16:20:56 -07:00 committed by GitHub
parent 6986f0d756
commit 18b5194f2e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 38 additions and 297 deletions

View File

@ -28,7 +28,6 @@ import (
"github.com/letsencrypt/boulder/metrics"
"github.com/letsencrypt/boulder/mocks"
"github.com/letsencrypt/boulder/ra"
"github.com/letsencrypt/boulder/rocsp"
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
"github.com/letsencrypt/boulder/sa"
sapb "github.com/letsencrypt/boulder/sa/proto"
@ -68,7 +67,7 @@ func TestRevokeBatch(t *testing.T) {
"../../test/hierarchy/int-r3.cert.pem": 102,
})
test.AssertNotError(t, err, "error loading issuers")
ssa, err := sa.NewSQLStorageAuthority(dbMap, dbMap, rocsp.NewMockWriteSucceedClient(), rocspIssuers, fc, log, metrics.NoopRegisterer, 1)
ssa, err := sa.NewSQLStorageAuthority(dbMap, dbMap, nil, rocspIssuers, fc, log, metrics.NoopRegisterer, 1)
if err != nil {
t.Fatalf("Failed to create SA: %s", err)
}
@ -515,7 +514,7 @@ func setup(t *testing.T) testCtx {
"../../test/hierarchy/int-r3.cert.pem": 102,
})
test.AssertNotError(t, err, "error loading issuers")
ssa, err := sa.NewSQLStorageAuthority(dbMap, dbMap, rocsp.NewMockWriteSucceedClient(), rocspIssuers, fc, log, metrics.NoopRegisterer, 1)
ssa, err := sa.NewSQLStorageAuthority(dbMap, dbMap, nil, rocspIssuers, fc, log, metrics.NoopRegisterer, 1)
if err != nil {
t.Fatalf("Failed to create SA: %s", err)
}

View File

@ -14,14 +14,12 @@ import (
"github.com/letsencrypt/boulder/db"
blog "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/ocsp/helper"
"golang.org/x/crypto/ocsp"
)
type client struct {
issuers []rocsp_config.ShortIDIssuer
redis *rocsp.WritingClient
db *db.WrappedMap // optional
ocspGenerator capb.OCSPGeneratorClient
@ -220,13 +218,8 @@ func (cl *client) signAndStoreResponses(ctx context.Context, input <-chan *sa.Ce
output <- processResult{id: uint64(status.ID), err: err}
continue
}
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())
err = cl.redis.StoreResponse(ctx, result.Response)
if err != nil {
output <- processResult{id: uint64(status.ID), err: err}
} else {
@ -263,16 +256,6 @@ func (cl *client) storeResponse(ctx context.Context, respBytes []byte) error {
if err != nil {
return fmt.Errorf("parsing response: %w", err)
}
issuer, err := rocsp_config.FindIssuerByName(resp, cl.issuers)
if err != nil {
return fmt.Errorf("finding issuer for response: %w", err)
}
// Re-parse the response, this time verifying with the appropriate issuer
resp, err = ocsp.ParseResponse(respBytes, issuer.Certificate.Certificate)
if err != nil {
return fmt.Errorf("parsing response: %w", err)
}
serial := core.SerialToString(resp.SerialNumber)
@ -289,7 +272,7 @@ func (cl *client) storeResponse(ctx context.Context, respBytes []byte) error {
time.Until(resp.NextUpdate).Hours(),
)
err = cl.redis.StoreResponse(ctx, respBytes, issuer.ShortID())
err = cl.redis.StoreResponse(ctx, respBytes)
if err != nil {
return fmt.Errorf("storing response: %w", err)
}
@ -299,7 +282,7 @@ func (cl *client) storeResponse(ctx context.Context, respBytes []byte) error {
return fmt.Errorf("getting response: %w", err)
}
parsedRetrievedResponse, err := ocsp.ParseResponse(retrievedResponse, issuer.Certificate.Certificate)
parsedRetrievedResponse, err := ocsp.ParseResponse(retrievedResponse, nil)
if err != nil {
return fmt.Errorf("parsing retrieved response: %w", err)
}

View File

@ -15,7 +15,6 @@ import (
blog "github.com/letsencrypt/boulder/log"
"github.com/letsencrypt/boulder/metrics"
"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"
@ -95,15 +94,7 @@ func TestStoreResponse(t *testing.T) {
}, issuerKey)
test.AssertNotError(t, err, "creating OCSP response")
issuers, err := rocsp_config.LoadIssuers(map[string]int{
"../../test/hierarchy/int-e1.cert.pem": 23,
})
if err != nil {
t.Fatal(err)
}
cl := client{
issuers: issuers,
redis: redisClient,
db: nil,
ocspGenerator: nil,
@ -147,7 +138,6 @@ func TestLoadFromDB(t *testing.T) {
}
rocspToolClient := client{
issuers: nil,
redis: redisClient,
db: dbMap,
ocspGenerator: mockOCSPGenerator{},

View File

@ -26,12 +26,6 @@ type Config struct {
ROCSPTool struct {
DebugAddr string
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
// If using load-from-db, this provides credentials to connect to the DB
// and the CA. Otherwise, it's optional.
@ -98,13 +92,6 @@ func main2() error {
defer logger.AuditPanic()
logger.Info(cmd.VersionString())
issuers, err := rocsp_config.LoadIssuers(c.ROCSPTool.Issuers)
if err != nil {
return fmt.Errorf("loading issuers: %w", err)
}
if len(issuers) == 0 {
return fmt.Errorf("'issuers' section of config JSON is required")
}
clk := cmd.Clock()
redisClient, err := rocsp_config.MakeClient(&c.ROCSPTool.Redis, clk, metrics.NoopRegisterer)
if err != nil {
@ -138,7 +125,6 @@ func main2() error {
ctx := context.Background()
cl := client{
issuers: issuers,
redis: redisClient,
db: db,
ocspGenerator: ocspGenerator,
@ -174,15 +160,6 @@ func main2() error {
if err != nil {
return fmt.Errorf("loading OCSP responses from DB: %w", err)
}
case "scan-metadata":
results := cl.redis.ScanMetadata(ctx, "*")
for r := range results {
if r.Err != nil {
cmd.FailOnError(err, "while scanning")
}
age := clk.Now().Sub(r.Metadata.ThisUpdate)
logger.Infof("%s: %g\n", r.Serial, age.Hours())
}
case "scan-responses":
results := cl.redis.ScanResponses(ctx, "*")
for r := range results {

View File

@ -12,7 +12,7 @@ type MockWriteClient struct {
// StoreResponse mocks a rocsp.StoreResponse method and returns nil or an
// error depending on the desired state.
func (r MockWriteClient) StoreResponse(ctx context.Context, respBytes []byte, shortIssuerID byte) error {
func (r MockWriteClient) StoreResponse(ctx context.Context, respBytes []byte) error {
return r.StoreReponseReturnError
}

View File

@ -2,7 +2,6 @@ package rocsp
import (
"context"
"encoding/binary"
"errors"
"fmt"
"strings"
@ -17,70 +16,6 @@ import (
var ErrRedisNotFound = errors.New("redis key not found")
// Metadata represents information stored with the 'm' prefix in the Redis DB:
// information required to maintain or serve the response, but not the response
// itself.
type Metadata struct {
ShortIssuerID byte
// ThisUpdate contains the ThisUpdate time of the stored OCSP response.
ThisUpdate time.Time
}
// String implements pretty-printing of Metadata
func (m Metadata) String() string {
return fmt.Sprintf("shortIssuerID: 0x%x, updated at: %s", m.ShortIssuerID, m.ThisUpdate)
}
// Marshal turns a metadata into a slice of 9 bytes for writing into Redis.
// Storing these always as 9 bytes gives us some potential to change the
// storage format non-disruptively in the future, so long as we can distinguish
// on the length of the stored value.
func (m Metadata) Marshal() []byte {
var output [9]byte
output[0] = m.ShortIssuerID
epochSeconds := uint64(m.ThisUpdate.Unix())
binary.LittleEndian.PutUint64(output[1:], epochSeconds)
return output[:]
}
// UnmarshalMetadata takes data from Redis and turns it into a Metadata object.
func UnmarshalMetadata(input []byte) (Metadata, error) {
if len(input) != 9 {
return Metadata{}, fmt.Errorf("invalid metadata length %d", len(input))
}
var output Metadata
output.ShortIssuerID = input[0]
epochSeconds := binary.LittleEndian.Uint64(input[1:])
output.ThisUpdate = time.Unix(int64(epochSeconds), 0).UTC()
return output, nil
}
// MakeResponseKey generates a Redis key string under which a response with the
// given serial should be stored.
func MakeResponseKey(serial string) string {
return fmt.Sprintf("r{%s}", serial)
}
// MakeMetadataKey generates a Redis key string under which metadata for the
// response with the given serial should be stored.
func MakeMetadataKey(serial string) string {
return fmt.Sprintf("m{%s}", serial)
}
func SerialFromResponseKey(key string) (string, error) {
if len(key) != 39 || key[0:2] != "r{" || key[38:39] != "}" {
return "", fmt.Errorf("malformed Redis OCSP response key %q", key)
}
return key[2:38], nil
}
func SerialFromMetadataKey(key string) (string, error) {
if len(key) != 39 || key[0:2] != "m{" || key[38:39] != "}" {
return "", fmt.Errorf("malformed Redis OCSP metadata key %q", key)
}
return key[2:38], nil
}
// Client represents a read-only Redis client.
type Client struct {
rdb *redis.ClusterClient
@ -109,9 +44,9 @@ func NewClient(
getLatency := prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Name: "rocsp_get_latency",
Help: "Histogram of latencies of rocsp.GetResponse and rocsp.GetMetadata calls with result and method labels",
Help: "Histogram of latencies of rocsp.GetResponse calls with result",
},
[]string{"result", "method"},
[]string{"result"},
)
stats.MustRegister(getLatency)
@ -143,12 +78,9 @@ func NewWritingClient(rdb *redis.ClusterClient, timeout time.Duration, clk clock
}
// StoreResponse parses the given bytes as an OCSP response, and stores it
// into Redis, updating both the metadata and response keys. ShortIssuerID
// is an arbitrarily assigned byte that unique identifies each issuer.
// Must be the same across OCSP components. Returns error if the OCSP
// response fails to parse. The expiration time (ttl) of the Redis key is
// set to OCSP response `NextUpdate`.
func (c *WritingClient) StoreResponse(ctx context.Context, respBytes []byte, shortIssuerID byte) error {
// into Redis. Returns error if the OCSP response fails to parse. The
// expiration time (ttl) of the Redis key is set to OCSP response `NextUpdate`.
func (c *WritingClient) StoreResponse(ctx context.Context, respBytes []byte) error {
start := c.clk.Now()
ctx, cancel := context.WithTimeout(ctx, c.timeout)
defer cancel()
@ -160,31 +92,10 @@ func (c *WritingClient) StoreResponse(ctx context.Context, respBytes []byte, sho
serial := core.SerialToString(resp.SerialNumber)
responseKey := MakeResponseKey(serial)
metadataKey := MakeMetadataKey(serial)
metadataStruct := Metadata{
ThisUpdate: resp.ThisUpdate,
ShortIssuerID: shortIssuerID,
}
metadataValue := metadataStruct.Marshal()
// Set the ttl duration to the response `NextUpdate - now()`
ttl := time.Until(resp.NextUpdate)
err = c.rdb.Watch(ctx, func(tx *redis.Tx) error {
err := tx.Set(ctx, responseKey, respBytes, ttl).Err()
if err != nil {
return fmt.Errorf("setting response: %w", err)
}
err = tx.Set(ctx, metadataKey, metadataValue, ttl).Err()
if err != nil {
return fmt.Errorf("setting metadata: %w", err)
}
return nil
}, metadataKey, responseKey)
err = c.rdb.Set(ctx, serial, respBytes, ttl).Err()
if err != nil {
state := "failed"
if errors.Is(err, context.DeadlineExceeded) {
@ -193,7 +104,7 @@ func (c *WritingClient) StoreResponse(ctx context.Context, respBytes []byte, sho
state = "canceled"
}
c.storeResponseLatency.With(prometheus.Labels{"result": state}).Observe(time.Since(start).Seconds())
return fmt.Errorf("transaction failed: %w", err)
return fmt.Errorf("setting response: %w", err)
}
c.storeResponseLatency.With(prometheus.Labels{"result": "success"}).Observe(time.Since(start).Seconds())
@ -202,20 +113,17 @@ func (c *WritingClient) StoreResponse(ctx context.Context, respBytes []byte, sho
// GetResponse fetches a response for the given serial number.
// Returns error if the OCSP response fails to parse.
// Does not check the metadata field.
func (c *Client) GetResponse(ctx context.Context, serial string) ([]byte, error) {
start := c.clk.Now()
ctx, cancel := context.WithTimeout(ctx, c.timeout)
defer cancel()
responseKey := MakeResponseKey(serial)
resp, err := c.rdb.Get(ctx, responseKey).Result()
resp, err := c.rdb.Get(ctx, serial).Result()
if err != nil {
// go-redis `Get` returns redis.Nil error when key does not exist. In
// that case return a `ErrRedisNotFound` error.
if errors.Is(err, redis.Nil) {
c.getLatency.With(prometheus.Labels{"result": "notFound", "method": "GetResponse"}).Observe(time.Since(start).Seconds())
c.getLatency.With(prometheus.Labels{"result": "notFound"}).Observe(time.Since(start).Seconds())
return nil, ErrRedisNotFound
}
@ -225,50 +133,14 @@ func (c *Client) GetResponse(ctx context.Context, serial string) ([]byte, error)
} else if errors.Is(err, context.Canceled) {
state = "canceled"
}
c.getLatency.With(prometheus.Labels{"result": state, "method": "GetResponse"}).Observe(time.Since(start).Seconds())
c.getLatency.With(prometheus.Labels{"result": state}).Observe(time.Since(start).Seconds())
return nil, fmt.Errorf("getting response: %w", err)
}
c.getLatency.With(prometheus.Labels{"result": "success", "method": "GetResponse"}).Observe(time.Since(start).Seconds())
c.getLatency.With(prometheus.Labels{"result": "success"}).Observe(time.Since(start).Seconds())
return []byte(resp), nil
}
// GetMetadata fetches the metadata for the given serial number.
func (c *Client) GetMetadata(ctx context.Context, serial string) (*Metadata, error) {
start := c.clk.Now()
ctx, cancel := context.WithTimeout(ctx, c.timeout)
defer cancel()
metadataKey := MakeMetadataKey(serial)
resp, err := c.rdb.Get(ctx, metadataKey).Result()
if err != nil {
// go-redis `Get` returns redis.Nil error when key does not exist. In
// that case return a `ErrRedisNotFound` error.
if errors.Is(err, redis.Nil) {
c.getLatency.With(prometheus.Labels{"result": "notFound", "method": "GetMetadata"}).Observe(time.Since(start).Seconds())
return nil, ErrRedisNotFound
}
state := "failed"
if errors.Is(err, context.DeadlineExceeded) {
state = "deadlineExceeded"
} else if errors.Is(err, context.Canceled) {
state = "canceled"
}
c.getLatency.With(prometheus.Labels{"result": state, "method": "GetMetadata"}).Observe(time.Since(start).Seconds())
return nil, fmt.Errorf("getting metadata: %w", err)
}
metadata, err := UnmarshalMetadata([]byte(resp))
if err != nil {
c.getLatency.With(prometheus.Labels{"result": "failed", "method": "GetMetadata"}).Observe(time.Since(start).Seconds())
return nil, fmt.Errorf("unmarshaling metadata: %w", err)
}
c.getLatency.With(prometheus.Labels{"result": "success", "method": "GetMetadata"}).Observe(time.Since(start).Seconds())
return &metadata, nil
}
// ScanResponsesResult represents a single OCSP response entry in redis.
// `Serial` is the stringified serial number of the response. `Body` is the
// DER bytes of the response. If this object represents an error, `Err` will
@ -291,17 +163,12 @@ func (c *Client) ScanResponses(ctx context.Context, serialPattern string) <-chan
iter := rdb.Scan(ctx, 0, pattern, 0).Iterator()
for iter.Next(ctx) {
key := iter.Val()
serial, err := SerialFromResponseKey(key)
if err != nil {
results <- ScanResponsesResult{Err: err}
continue
}
val, err := c.rdb.Get(ctx, key).Result()
if err != nil {
results <- ScanResponsesResult{Err: fmt.Errorf("getting metadata: %w", err)}
results <- ScanResponsesResult{Err: fmt.Errorf("getting response: %w", err)}
continue
}
results <- ScanResponsesResult{Serial: serial, Body: []byte(val)}
results <- ScanResponsesResult{Serial: key, Body: []byte(val)}
}
return iter.Err()
})
@ -312,51 +179,3 @@ func (c *Client) ScanResponses(ctx context.Context, serialPattern string) <-chan
}()
return results
}
// ScanMetadataResult represents a single OCSP response entry in redis.
// `Serial` is the stringified serial number of the response. `Metadata` is the
// parsed metadata. If this object represents an error, `Err` will
// be non-nil and the other entries will have their zero values.
type ScanMetadataResult struct {
Serial string
Metadata *Metadata
Err error
}
// ScanMetadata scans Redis for the metadata of all OCSP responses where the serial number matches
// the provided pattern. It returns immediately and emits results and errors on
// `<-chan ScanResponsesResult`. It closes the channel when it is done or hits an error.
func (c *Client) ScanMetadata(ctx context.Context, serialPattern string) <-chan ScanMetadataResult {
pattern := fmt.Sprintf("m{%s}", serialPattern)
results := make(chan ScanMetadataResult)
go func() {
defer close(results)
var cursor uint64
for {
var keys []string
var err error
keys, cursor, err = c.rdb.Scan(ctx, cursor, pattern, 10).Result()
if err != nil {
results <- ScanMetadataResult{Err: err}
return
}
if cursor == 0 {
return
}
for _, key := range keys {
serial, err := SerialFromMetadataKey(key)
if err != nil {
results <- ScanMetadataResult{Err: err}
return
}
m, err := c.GetMetadata(ctx, serial)
if err != nil {
results <- ScanMetadataResult{Err: err}
return
}
results <- ScanMetadataResult{Serial: serial, Metadata: m}
}
}
}()
return results
}

View File

@ -44,8 +44,7 @@ func TestSetAndGet(t *testing.T) {
if err != nil {
t.Fatal(err)
}
var shortIssuerID byte = 99
err = client.StoreResponse(context.Background(), response, shortIssuerID)
err = client.StoreResponse(context.Background(), response)
if err != nil {
t.Fatalf("storing response: %s", err)
}
@ -58,19 +57,4 @@ func TestSetAndGet(t *testing.T) {
if !bytes.Equal(resp2, response) {
t.Errorf("response written and response retrieved were not equal")
}
metadata, err := client.GetMetadata(context.Background(), serial)
if err != nil {
t.Fatalf("getting metadata: %s", err)
}
if metadata.ShortIssuerID != shortIssuerID {
t.Errorf("expected shortIssuerID %d, got %d", shortIssuerID, metadata.ShortIssuerID)
}
expectedTime, err := time.Parse(time.RFC3339, "2021-10-25T20:00:00Z")
if err != nil {
t.Fatalf("failed to parse time: %s", err)
}
if metadata.ThisUpdate != expectedTime {
t.Errorf("expected ThisUpdate %q, got %q", expectedTime, metadata.ThisUpdate)
}
}

View File

@ -165,7 +165,7 @@ func (ssa *SQLStorageAuthority) AddPrecertificate(ctx context.Context, req *sapb
// Send the response off to redis in a goroutine.
go func() {
err = ssa.storeOCSPRedis(rocspCtx, req.Ocsp, req.IssuerID)
err = ssa.storeOCSPRedis(rocspCtx, req.Ocsp)
ssa.log.Debugf("failed to store OCSP response in redis: %v", err)
}()
}

View File

@ -2,22 +2,15 @@ package sa
import (
"context"
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
)
type rocspWriter interface {
StoreResponse(ctx context.Context, respBytes []byte, shortIssuerID byte) error
StoreResponse(ctx context.Context, respBytes []byte) error
}
// storeOCSPRedis stores an OCSP response in a redis cluster.
func (ssa *SQLStorageAuthority) storeOCSPRedis(ctx context.Context, resp []byte, issuerID int64) error {
shortIssuerID, err := rocsp_config.FindIssuerByID(issuerID, ssa.shortIssuers)
if err != nil {
ssa.redisStoreResponse.WithLabelValues("find_issuer_error").Inc()
return err
}
err = ssa.rocspWriteClient.StoreResponse(ctx, resp, shortIssuerID.ShortID())
func (ssa *SQLStorageAuthority) storeOCSPRedis(ctx context.Context, resp []byte) error {
err := ssa.rocspWriteClient.StoreResponse(ctx, resp)
if err != nil {
ssa.redisStoreResponse.WithLabelValues("store_response_error").Inc()
return err

View File

@ -19,21 +19,10 @@ func TestStoreOCSPRedis(t *testing.T) {
response, err := getOCSPResponse()
test.AssertNotError(t, err, "unexpected error")
ctx := context.Background()
err = sa.storeOCSPRedis(ctx, response, 58923463773186183)
err = sa.storeOCSPRedis(ctx, response)
test.AssertNotError(t, err, "unexpected error")
}
func TestStoreOCSPRedisInvalidIssuer(t *testing.T) {
sa, _, cleanUp := initSA(t)
defer cleanUp()
response, err := getOCSPResponse()
test.AssertNotError(t, err, "unexpected error")
ctx := context.Background()
// 1234 is expected to not be a valid issuerID
err = sa.storeOCSPRedis(ctx, response, 1234)
test.AssertContains(t, err.Error(), "no issuer found for an ID in certificateStatus: 1234")
}
func TestStoreOCSPRedisFail(t *testing.T) {
sa, _, cleanUp := initSA(t)
defer cleanUp()
@ -41,6 +30,6 @@ func TestStoreOCSPRedisFail(t *testing.T) {
response, err := getOCSPResponse()
test.AssertNotError(t, err, "unexpected error")
ctx := context.Background()
err = sa.storeOCSPRedis(ctx, response, 58923463773186183)
err = sa.storeOCSPRedis(ctx, response)
test.AssertContains(t, err.Error(), "could not store response")
}

View File

@ -29,6 +29,7 @@ import (
"github.com/letsencrypt/boulder/identifier"
blog "github.com/letsencrypt/boulder/log"
"github.com/letsencrypt/boulder/revocation"
"github.com/letsencrypt/boulder/rocsp"
rocsp_config "github.com/letsencrypt/boulder/rocsp/config"
sapb "github.com/letsencrypt/boulder/sa/proto"
)
@ -93,7 +94,7 @@ type orderFQDNSet struct {
func NewSQLStorageAuthority(
dbMap *db.WrappedMap,
dbReadOnlyMap *db.WrappedMap,
rocspWriteClient rocspWriter,
rocspWriteClient *rocsp.WritingClient,
shortIssuers []rocsp_config.ShortIDIssuer,
clk clock.Clock,
logger blog.Logger,
@ -114,10 +115,15 @@ func NewSQLStorageAuthority(
}, []string{"result"})
stats.MustRegister(redisStoreResponse)
var rocspWriter rocspWriter
if rocspWriteClient != nil {
rocspWriter = rocspWriteClient
}
ssa := &SQLStorageAuthority{
dbMap: dbMap,
dbReadOnlyMap: dbReadOnlyMap,
rocspWriteClient: rocspWriteClient,
rocspWriteClient: rocspWriter,
shortIssuers: shortIssuers,
clk: clk,
log: logger,
@ -1776,7 +1782,7 @@ func (ssa *SQLStorageAuthority) RevokeCertificate(ctx context.Context, req *sapb
// Send the response off to redis in a goroutine.
go func() {
err = ssa.storeOCSPRedis(rocspCtx, req.Response, req.IssuerID)
err = ssa.storeOCSPRedis(rocspCtx, req.Response)
ssa.log.Debugf("failed to store OCSP response in redis: %v", err)
}()
}
@ -1836,7 +1842,7 @@ func (ssa *SQLStorageAuthority) UpdateRevokedCertificate(ctx context.Context, re
// Send the response off to redis in a goroutine.
go func() {
err = ssa.storeOCSPRedis(rocspCtx, req.Response, req.IssuerID)
err = ssa.storeOCSPRedis(rocspCtx, req.Response)
ssa.log.Debugf("failed to store OCSP response in redis: %v", err)
}()
}

View File

@ -82,10 +82,11 @@ func initSA(t *testing.T) (*SQLStorageAuthority, clock.FakeClock, func()) {
if err != nil {
t.Fatalf("failed to load issuers: %s", err)
}
sa, err := NewSQLStorageAuthority(dbMap, dbMap, rocsp.NewMockWriteSucceedClient(), rocspIssuers, fc, log, metrics.NoopRegisterer, 1)
sa, err := NewSQLStorageAuthority(dbMap, dbMap, nil, rocspIssuers, fc, log, metrics.NoopRegisterer, 1)
if err != nil {
t.Fatalf("Failed to create SA: %s", err)
}
sa.rocspWriteClient = rocsp.NewMockWriteSucceedClient()
cleanUp := test.ResetSATestDatabase(t)
return sa, fc, cleanUp