Support for generating Delta CRLs (#16773)

* Allow generation of up-to-date delta CRLs

While switching to periodic rebuilds of CRLs alleviates the constant
rebuild pressure on Vault during times of high revocation, the CRL
proper becomes stale. One response to this is to switch to OCSP, but not
every system has support for this. Additionally, OCSP usually requires
connectivity and isn't used to augment a pre-distributed CRL (and is
instead used independently).

By generating delta CRLs containing only new revocations, an existing
CRL can be supplemented with newer revocations without requiring Vault
to rebuild all complete CRLs. Admins can periodically fetch the delta
CRL and add it to the existing CRL and applications should be able to
support using serials from both.

Because delta CRLs are emptied when the next complete CRL is rebuilt, it
is important that applications fetch the delta CRL and correlate it to
their complete CRL; if their complete CRL is older than the delta CRL's
extension number, applications MUST fetch the newer complete CRL to
ensure they have a correct combination.

This modifies the revocation process and adds several new configuration
options, controlling whether Delta CRLs are enabled and when we'll
rebuild it.

Signed-off-by: Alexander Scheel <alex.scheel@hashicorp.com>

* Add tests for delta CRLs

Signed-off-by: Alexander Scheel <alex.scheel@hashicorp.com>

* Add changelog entry

Signed-off-by: Alexander Scheel <alex.scheel@hashicorp.com>

* Add documentation on delta CRLs

Signed-off-by: Alexander Scheel <alex.scheel@hashicorp.com>

* Address review feedback: fix several bugs

Thanks Steve!

Signed-off-by: Alexander Scheel <alex.scheel@hashicorp.com>

* Correctly invoke periodic func on active nodes

We need to ensure we read the updated config (in case of OCSP request
handling on standby nodes), but otherwise want to avoid CRL/DeltaCRL
re-building.

Signed-off-by: Alexander Scheel <alex.scheel@hashicorp.com>

Signed-off-by: Alexander Scheel <alex.scheel@hashicorp.com>
This commit is contained in:
Alexander Scheel 2022-08-29 11:37:09 -04:00 committed by GitHub
parent 8c9c1d2b2a
commit e03fb14be4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 641 additions and 72 deletions

View File

@ -75,11 +75,16 @@ func Backend(conf *logical.BackendConfig) *backend {
"ca/pem",
"ca_chain",
"ca",
"crl/delta",
"crl/delta/pem",
"crl/pem",
"crl",
"issuer/+/crl/der",
"issuer/+/crl/pem",
"issuer/+/crl",
"issuer/+/crl/delta/der",
"issuer/+/crl/delta/pem",
"issuer/+/crl/delta",
"issuer/+/pem",
"issuer/+/der",
"issuer/+/json",
@ -89,7 +94,8 @@ func Backend(conf *logical.BackendConfig) *backend {
},
LocalStorage: []string{
"revoked/",
revokedPath,
deltaWALPath,
legacyCRLPath,
"crls/",
"certs/",
@ -398,6 +404,13 @@ func (b *backend) periodicFunc(ctx context.Context, request *logical.Request) er
return err
}
// As we're (below) modifying the backing storage, we need to ensure
// we're not on a standby/secondary node.
if b.System().ReplicationState().HasState(consts.ReplicationPerformanceStandby) ||
b.System().ReplicationState().HasState(consts.ReplicationDRSecondary) {
return nil
}
// Check if we're set to auto rebuild and a CRL is set to expire.
if err := b.crlBuilder.checkForAutoRebuild(sc); err != nil {
return err
@ -408,6 +421,13 @@ func (b *backend) periodicFunc(ctx context.Context, request *logical.Request) er
return err
}
// If a delta CRL was rebuilt above as part of the complete CRL rebuild,
// this will be a no-op. However, if we do need to rebuild delta CRLs,
// this would cause us to do so.
if err := b.crlBuilder.rebuildDeltaCRLsIfForced(sc); err != nil {
return err
}
// All good!
return nil
}

View File

@ -174,7 +174,7 @@ func fetchCertBySerial(ctx context.Context, b *backend, req *logical.Request, pr
case strings.HasPrefix(prefix, "revoked/"):
legacyPath = "revoked/" + colonSerial
path = "revoked/" + hyphenSerial
case serial == legacyCRLPath:
case serial == legacyCRLPath || serial == deltaCRLPath:
if err = b.crlBuilder.rebuildIfForced(ctx, b, req); err != nil {
return nil, err
}
@ -183,6 +183,14 @@ func fetchCertBySerial(ctx context.Context, b *backend, req *logical.Request, pr
if err != nil {
return nil, err
}
if serial == deltaCRLPath {
if sc.Backend.useLegacyBundleCaStorage() {
return nil, fmt.Errorf("refusing to serve delta CRL with legacy CA bundle")
}
path += deltaCRLPathSuffix
}
default:
legacyPath = "certs/" + colonSerial
path = "certs/" + hyphenSerial

View File

@ -285,6 +285,14 @@ func crlEnableDisableTestForBackend(t *testing.T, b *backend, s logical.Storage,
for _, serialNum := range expectedSerials {
requireSerialNumberInCRL(t, certList, serialNum)
}
// Since this test assumes a complete CRL was rebuilt, we can grab
// the delta CRL and ensure it is empty.
deltaList := getParsedCrlFromBackend(t, b, s, "crl/delta").TBSCertList
lenDeltaList := len(deltaList.RevokedCertificates)
if lenDeltaList != 0 {
t.Fatalf("expected zero revoked certificates on the delta CRL due to complete CRL rebuild, found %d", lenDeltaList)
}
}
revoke := func(serialIndex int) {
@ -378,7 +386,7 @@ func TestCrlRebuilder(t *testing.T) {
require.NoError(t, err)
req := &logical.Request{Storage: s}
cb := crlBuilder{}
cb := newCRLBuilder()
// Force an initial build
err = cb.rebuild(ctx, b, req, true)
@ -848,9 +856,14 @@ func TestAutoRebuild(t *testing.T) {
// the rollback manager timer ticks. With the new helper, we can
// modify the rollback manager timer period directly, allowing us
// to shorten the total test time significantly.
newPeriod := 6 * time.Second
gracePeriod := (newPeriod + 1*time.Second).String()
crlTime := (newPeriod + 2*time.Second).String()
//
// We set the delta CRL time to ensure it executes prior to the
// main CRL rebuild, and the new CRL doesn't rebuild until after
// we're done.
newPeriod := 1 * time.Second
deltaPeriod := (newPeriod + 1*time.Second).String()
crlTime := (6*newPeriod + 2*time.Second).String()
gracePeriod := (3 * newPeriod).String()
delta := 2 * newPeriod
// This test requires the periodicFunc to trigger, which requires we stand
@ -913,9 +926,10 @@ func TestAutoRebuild(t *testing.T) {
require.Equal(t, resp.Data["ocsp_disable"], defaultCrlConfig.OcspDisable)
require.Equal(t, resp.Data["auto_rebuild"], defaultCrlConfig.AutoRebuild)
require.Equal(t, resp.Data["auto_rebuild_grace_period"], defaultCrlConfig.AutoRebuildGracePeriod)
require.Equal(t, resp.Data["enable_delta"], defaultCrlConfig.EnableDelta)
require.Equal(t, resp.Data["delta_rebuild_interval"], defaultCrlConfig.DeltaRebuildInterval)
// Safety guard: we play with rebuild timing below. We don't expect
// this entire test to take longer than 80s.
// Safety guard: we play with rebuild timing below.
_, err = client.Logical().Write("pki/config/crl", map[string]interface{}{
"expiry": crlTime,
})
@ -946,6 +960,8 @@ func TestAutoRebuild(t *testing.T) {
"expiry": crlTime,
"auto_rebuild": true,
"auto_rebuild_grace_period": gracePeriod,
"enable_delta": true,
"delta_rebuild_interval": deltaPeriod,
})
require.NoError(t, err)
@ -976,7 +992,7 @@ func TestAutoRebuild(t *testing.T) {
require.NotEmpty(t, resp.Data["uuid"])
pkiMount := resp.Data["uuid"].(string)
require.NotEmpty(t, pkiMount)
revEntryPath := "logical/" + pkiMount + "/" + revokedPath + strings.ReplaceAll(newLeafSerial, ":", "-")
revEntryPath := "logical/" + pkiMount + "/" + revokedPath + normalizeSerial(newLeafSerial)
// storage from cluster.Core[0] is a physical storage copy, not a logical
// storage. This difference means, if we were to do a storage.Get(...)
@ -989,17 +1005,15 @@ func TestAutoRebuild(t *testing.T) {
require.NotNil(t, resp.Data)
require.NotEmpty(t, resp.Data["value"])
revEntryValue := resp.Data["value"].(string)
fmt.Println(resp)
var revInfo revocationInfo
err = json.Unmarshal([]byte(revEntryValue), &revInfo)
require.NoError(t, err)
require.Equal(t, revInfo.CertificateIssuer, issuerID(rootIssuer))
// Serial should not appear on CRL.
// New serial should not appear on CRL.
crl = getCrlCertificateList(t, client, "pki")
thisCRLNumber := crl.Version
requireSerialNumberInCRL(t, crl, leafSerial)
requireSerialNumberInCRL(t, crl, leafSerial) // But the old one should.
now := time.Now()
graceInterval, _ := time.ParseDuration(gracePeriod)
expectedUpdate := lastCRLExpiry.Add(-1 * graceInterval)
@ -1018,6 +1032,79 @@ func TestAutoRebuild(t *testing.T) {
t.Fatalf("shouldn't be here")
}
// This serial should exist in the delta WAL section for the mount...
resp, err = client.Logical().List("sys/raw/logical/" + pkiMount + "/" + deltaWALPath)
require.NoError(t, err)
require.NotNil(t, resp)
require.NotEmpty(t, resp.Data)
require.NotEmpty(t, resp.Data["keys"])
require.Contains(t, resp.Data["keys"], normalizeSerial(newLeafSerial))
haveUpdatedDeltaCRL := false
interruptChan := time.After(4*newPeriod + delta)
for {
if haveUpdatedDeltaCRL {
break
}
select {
case <-interruptChan:
t.Fatalf("expected to regenerate delta CRL within a couple of periodicFunc invocations (plus %v grace period)", delta)
default:
// Check and see if there's a storage entry for the last rebuild
// serial. If so, validate the delta CRL contains this entry.
resp, err = client.Logical().List("sys/raw/logical/" + pkiMount + "/" + deltaWALPath)
require.NoError(t, err)
require.NotNil(t, resp)
require.NotEmpty(t, resp.Data)
require.NotEmpty(t, resp.Data["keys"])
haveRebuildMarker := false
for _, rawEntry := range resp.Data["keys"].([]interface{}) {
entry := rawEntry.(string)
if entry == deltaWALLastRevokedSerialName {
haveRebuildMarker = true
break
}
}
if !haveRebuildMarker {
time.Sleep(1 * time.Second)
continue
}
// Read the marker and see if its correct.
resp, err = client.Logical().Read("sys/raw/logical/" + pkiMount + "/" + deltaWALLastBuildSerial)
require.NoError(t, err)
if resp == nil {
time.Sleep(1 * time.Second)
continue
}
require.NotNil(t, resp)
require.NotEmpty(t, resp.Data)
require.NotEmpty(t, resp.Data["value"])
// Easy than JSON decoding...
if !strings.Contains(resp.Data["value"].(string), newLeafSerial) {
time.Sleep(1 * time.Second)
continue
}
haveUpdatedDeltaCRL = true
// Ensure it has what we want.
deltaCrl := getParsedCrlAtPath(t, client, "/v1/pki/crl/delta").TBSCertList
if !requireSerialNumberInCRL(nil, deltaCrl, newLeafSerial) {
// Check if it is on the main CRL because its already regenerated.
mainCRL := getParsedCrlAtPath(t, client, "/v1/pki/crl").TBSCertList
requireSerialNumberInCRL(t, mainCRL, newLeafSerial)
}
break
}
}
// Now, wait until we're within the grace period... Then start prompting
// for regeneration.
if expectedUpdate.After(now) {
@ -1026,7 +1113,7 @@ func TestAutoRebuild(t *testing.T) {
// Otherwise, the absolute latest we're willing to wait is some delta
// after CRL expiry (to let stuff regenerate &c).
interruptChan := time.After(lastCRLExpiry.Sub(now) + delta)
interruptChan = time.After(lastCRLExpiry.Sub(now) + delta)
for {
select {
case <-interruptChan:

View File

@ -10,18 +10,25 @@ import (
"math/big"
"strings"
"sync"
"sync/atomic"
"time"
atomic2 "go.uber.org/atomic"
"github.com/hashicorp/vault/sdk/helper/consts"
"github.com/hashicorp/vault/sdk/helper/certutil"
"github.com/hashicorp/vault/sdk/helper/errutil"
"github.com/hashicorp/vault/sdk/logical"
)
const revokedPath = "revoked/"
const (
revokedPath = "revoked/"
deltaWALPath = "delta-wal/"
deltaWALLastBuildSerialName = "last-build-serial"
deltaWALLastBuildSerial = deltaWALPath + deltaWALLastBuildSerialName
deltaWALLastRevokedSerialName = "last-revoked-serial"
deltaWALLastRevokedSerial = deltaWALPath + deltaWALLastRevokedSerialName
)
type revocationInfo struct {
CertificateBytes []byte `json:"certificate_bytes"`
@ -30,6 +37,29 @@ type revocationInfo struct {
CertificateIssuer issuerID `json:"issuer_id"`
}
type (
// Placeholder in case of migrations needing more data. Currently
// we use the path name to store the serial number that was revoked.
deltaWALInfo struct{}
lastWALInfo struct {
// Info to write about the last WAL entry. This is the serial number
// of the last revoked certificate.
//
// We write this below in revokedCert(...) and read it in
// rebuildDeltaCRLsIfForced(...).
Serial string `json:"serial"`
}
lastDeltaInfo struct {
// Info to write about the last built delta CRL. This is the serial
// number of the last revoked certificate that we saw prior to delta
// CRL building.
//
// We write this below in buildAnyCRLs(...) and read it in
// rebuildDeltaCRLsIfForced(...).
Serial string `json:"serial"`
}
)
// crlBuilder is gatekeeper for controlling various read/write operations to the storage of the CRL.
// The extra complexity arises from secondary performance clusters seeing various writes to its storage
// without the actual API calls. During the storage invalidation process, we do not have the required state
@ -38,12 +68,13 @@ type revocationInfo struct {
//
// The CRL builder also tracks the revocation configuration.
type crlBuilder struct {
m sync.Mutex
forceRebuild uint32
_builder sync.Mutex
forceRebuild *atomic2.Bool
lastDeltaRebuildCheck time.Time
c sync.RWMutex
dirty *atomic2.Bool
config crlConfig
_config sync.RWMutex
dirty *atomic2.Bool
config crlConfig
}
const (
@ -53,8 +84,13 @@ const (
func newCRLBuilder() *crlBuilder {
return &crlBuilder{
dirty: atomic2.NewBool(true),
config: defaultCrlConfig,
forceRebuild: atomic2.NewBool(false),
// Set the last delta rebuild window to now, delaying the first delta
// rebuild by the first rebuild period to give us some time on startup
// to stabilize.
lastDeltaRebuildCheck: time.Now(),
dirty: atomic2.NewBool(true),
config: defaultCrlConfig,
}
}
@ -65,8 +101,8 @@ func (cb *crlBuilder) markConfigDirty() {
func (cb *crlBuilder) reloadConfigIfRequired(sc *storageContext) error {
if cb.dirty.Load() {
// Acquire a write lock.
cb.c.Lock()
defer cb.c.Unlock()
cb._config.Lock()
defer cb._config.Unlock()
if !cb.dirty.Load() {
// Someone else might've been reloading the config; no need
@ -100,8 +136,8 @@ func (cb *crlBuilder) getConfigWithUpdate(sc *storageContext) (*crlConfig, error
return nil, err
}
cb.c.RLock()
defer cb.c.RUnlock()
cb._config.RLock()
defer cb._config.RUnlock()
configCopy := cb.config
return &configCopy, nil
@ -113,7 +149,7 @@ func (cb *crlBuilder) checkForAutoRebuild(sc *storageContext) error {
return err
}
if cfg.Disable || !cfg.AutoRebuild || atomic.LoadUint32(&cb.forceRebuild) == 1 {
if cfg.Disable || !cfg.AutoRebuild || cb.forceRebuild.Load() {
// Not enabled, not on auto-rebuilder, or we're already scheduled to
// rebuild so there's no point to interrogate CRL values...
return nil
@ -133,14 +169,14 @@ func (cb *crlBuilder) checkForAutoRebuild(sc *storageContext) error {
// If there's no config, assume we've gotta rebuild it to get this
// information.
if crlConfig == nil {
atomic.CompareAndSwapUint32(&cb.forceRebuild, 0, 1)
cb.forceRebuild.Store(true)
return nil
}
// If the map is empty, assume we need to upgrade and schedule a
// rebuild.
if len(crlConfig.CRLExpirationMap) == 0 {
atomic.CompareAndSwapUint32(&cb.forceRebuild, 0, 1)
cb.forceRebuild.Store(true)
return nil
}
@ -163,7 +199,7 @@ func (cb *crlBuilder) checkForAutoRebuild(sc *storageContext) error {
for _, value := range crlConfig.CRLExpirationMap {
if value.IsZero() || now.After(value.Add(-1*period)) {
atomic.CompareAndSwapUint32(&cb.forceRebuild, 0, 1)
cb.forceRebuild.Store(true)
return nil
}
}
@ -174,7 +210,7 @@ func (cb *crlBuilder) checkForAutoRebuild(sc *storageContext) error {
// rebuildIfForced is to be called by readers or periodic functions that might need to trigger
// a refresh of the CRL before the read occurs.
func (cb *crlBuilder) rebuildIfForced(ctx context.Context, b *backend, request *logical.Request) error {
if atomic.LoadUint32(&cb.forceRebuild) == 1 {
if cb.forceRebuild.Load() {
return cb._doRebuild(ctx, b, request, true, _enforceForceFlag)
}
@ -199,29 +235,163 @@ func (cb *crlBuilder) requestRebuildIfActiveNode(b *backend) {
b.Logger().Info("Scheduling PKI CRL rebuild.")
// Set the flag to 1, we don't care if we aren't the ones that actually swap it to 1.
atomic.CompareAndSwapUint32(&cb.forceRebuild, 0, 1)
cb.forceRebuild.Store(true)
}
func (cb *crlBuilder) _doRebuild(ctx context.Context, b *backend, request *logical.Request, forceNew bool, ignoreForceFlag bool) error {
cb.m.Lock()
defer cb.m.Unlock()
cb._builder.Lock()
defer cb._builder.Unlock()
// Re-read the lock in case someone beat us to the punch between the previous load op.
forceBuildFlag := atomic.LoadUint32(&cb.forceRebuild)
if forceBuildFlag == 1 || ignoreForceFlag {
forceBuildFlag := cb.forceRebuild.Load()
if forceBuildFlag || ignoreForceFlag {
// Reset our original flag back to 0 before we start the rebuilding. This may lead to another round of
// CRL building, but we want to avoid the race condition caused by clearing the flag after we completed (An
// update/revocation occurred attempting to set the flag, after we listed the certs but before we wrote
// the CRL, so we missed the update and cleared the flag.)
atomic.CompareAndSwapUint32(&cb.forceRebuild, 1, 0)
cb.forceRebuild.Store(false)
// if forceRebuild was requested, that should force a complete rebuild even if requested not too by forceNew
myForceNew := forceBuildFlag == 1 || forceNew
myForceNew := forceBuildFlag || forceNew
return buildCRLs(ctx, b, request, myForceNew)
}
return nil
}
func (cb *crlBuilder) getPresentDeltaWALForClearing(sc *storageContext) ([]string, error) {
// Clearing of the delta WAL occurs after a new complete CRL has been built.
walSerials, err := sc.Storage.List(sc.Context, deltaWALPath)
if err != nil {
return nil, fmt.Errorf("error fetching list of delta WAL certificates to clear: %s", err)
}
// We _should_ remove the special WAL entries here, but we don't really
// want to traverse the list again (and also below in clearDeltaWAL). So
// trust the latter does the right thing.
return walSerials, nil
}
func (cb *crlBuilder) clearDeltaWAL(sc *storageContext, walSerials []string) error {
// Clearing of the delta WAL occurs after a new complete CRL has been built.
for _, serial := range walSerials {
// Don't remove our special entries!
if serial == deltaWALLastBuildSerialName || serial == deltaWALLastRevokedSerialName {
continue
}
if err := sc.Storage.Delete(sc.Context, deltaWALPath+serial); err != nil {
return fmt.Errorf("error clearing delta WAL certificate: %s", err)
}
}
return nil
}
func (cb *crlBuilder) rebuildDeltaCRLsIfForced(sc *storageContext) error {
// Delta CRLs use the same expiry duration as the complete CRL. Because
// we always rebuild the complete CRL and then the delta CRL, we can
// be assured that the delta CRL always expires after a complete CRL,
// and that rebuilding the complete CRL will trigger a fresh delta CRL
// build of its own.
//
// This guarantee means we can avoid checking delta CRL expiry. Thus,
// we only need to rebuild the delta CRL when we have new revocations,
// within our time window for updating it.
cfg, err := cb.getConfigWithUpdate(sc)
if err != nil {
return err
}
if !cfg.EnableDelta {
// We explicitly do not update the last check time here, as we
// want to persist the last rebuild window if it hasn't been set.
return nil
}
deltaRebuildDuration, err := time.ParseDuration(cfg.DeltaRebuildInterval)
if err != nil {
return err
}
// Acquire CRL building locks before we get too much further.
cb._builder.Lock()
defer cb._builder.Unlock()
// Last is setup during newCRLBuilder(...), so we don't need to deal with
// a zero condition.
now := time.Now()
last := cb.lastDeltaRebuildCheck
nextRebuildCheck := last.Add(deltaRebuildDuration)
if now.Before(nextRebuildCheck) {
// If we're still before the time of our next rebuild check, we can
// safely return here even if we have certs. We'll wait for a bit,
// retrigger this check, and then do the rebuild.
return nil
}
// Update our check time. If we bail out below (due to storage errors
// or whatever), we'll delay the next CRL check (hopefully allowing
// things to stabilize). Otherwise, we might not build a new Delta CRL
// until our next complete CRL build.
cb.lastDeltaRebuildCheck = now
// Fetch two storage entries to see if we actually need to do this
// rebuild, given we're within the window.
lastWALEntry, err := sc.Storage.Get(sc.Context, deltaWALLastRevokedSerial)
if err != nil || lastWALEntry == nil || lastWALEntry.Value == nil {
// If this entry does not exist, we don't need to rebuild the
// delta WAL due to the expiration assumption above. There must
// not have been any new revocations. Since err should be nil
// in this case, we can safely return it.
return err
}
lastBuildEntry, err := sc.Storage.Get(sc.Context, deltaWALLastBuildSerial)
if err != nil {
return err
}
if lastBuildEntry != nil && lastBuildEntry.Value != nil {
// If the last build entry doesn't exist, we still want to build a
// new delta WAL, since this could be our very first time doing so.
//
// Otherwise, here, now that we know it exists, we want to check this
// value against the other value. Since we previously guarded the WAL
// entry being non-empty, we're good to decode everything within this
// guard.
var walInfo lastWALInfo
if err := lastWALEntry.DecodeJSON(&walInfo); err != nil {
return err
}
var deltaInfo lastDeltaInfo
if err := lastBuildEntry.DecodeJSON(&deltaInfo); err != nil {
return err
}
// Here, everything decoded properly and we know that no new certs
// have been revoked since we built this last delta CRL. We can exit
// without rebuilding then.
if walInfo.Serial == deltaInfo.Serial {
return nil
}
}
// Finally, we must've needed to do the rebuild. Execute!
return cb.rebuildDeltaCRLsHoldingLock(sc, false)
}
func (cb *crlBuilder) rebuildDeltaCRLs(sc *storageContext, forceNew bool) error {
cb._builder.Lock()
defer cb._builder.Unlock()
return cb.rebuildDeltaCRLsHoldingLock(sc, forceNew)
}
func (cb *crlBuilder) rebuildDeltaCRLsHoldingLock(sc *storageContext, forceNew bool) error {
return buildAnyCRLs(sc, forceNew, true /* building delta */)
}
// Helper function to fetch a map of issuerID->parsed cert for revocation
// usage. Unlike other paths, this needs to handle the legacy bundle
// more gracefully than rejecting it outright.
@ -390,6 +560,10 @@ func revokeCert(ctx context.Context, b *backend, req *logical.Request, serial st
}
if !config.AutoRebuild {
// Note that writing the Delta WAL here isn't necessary; we've
// already rebuilt the full CRL so the Delta WAL will be cleared
// afterwards. Writing an entry only to immediately remove it
// isn't necessary.
crlErr := b.crlBuilder.rebuild(ctx, b, req, false)
if crlErr != nil {
switch crlErr.(type) {
@ -399,6 +573,42 @@ func revokeCert(ctx context.Context, b *backend, req *logical.Request, serial st
return nil, fmt.Errorf("error encountered during CRL building: %w", crlErr)
}
}
} else if !alreadyRevoked {
// Regardless of whether or not we've presently enabled Delta CRLs,
// we should always write the Delta WAL in case it is enabled in the
// future. We could trigger another full CRL rebuild instead (to avoid
// inconsistent state between the CRL and missing Delta WAL entries),
// but writing extra (unused?) WAL entries versus an expensive full
// CRL rebuild is probably a net wash.
///
// We should only do this when the cert hasn't already been revoked.
// Otherwise, the re-revocation may appear on both an existing CRL and
// on a delta CRL, or a serial may be skipped from the delta CRL if
// there's an A->B->A revocation pattern and the delta was rebuilt
// after the first cert.
//
// Currently we don't store any data in the WAL entry.
var walInfo deltaWALInfo
walEntry, err := logical.StorageEntryJSON(deltaWALPath+normalizeSerial(serial), walInfo)
if err != nil {
return nil, fmt.Errorf("unable to create delta CRL WAL entry")
}
if err = req.Storage.Put(ctx, walEntry); err != nil {
return nil, fmt.Errorf("error saving delta CRL WAL entry")
}
// In order for periodic delta rebuild to be mildly efficient, we
// should write the last revoked delta WAL entry so we know if we
// have new revocations that we should rebuild the delta WAL for.
lastRevSerial := lastWALInfo{Serial: serial}
lastWALEntry, err := logical.StorageEntryJSON(deltaWALLastRevokedSerial, lastRevSerial)
if err != nil {
return nil, fmt.Errorf("unable to create last delta CRL WAL entry")
}
if err = req.Storage.Put(ctx, lastWALEntry); err != nil {
return nil, fmt.Errorf("error saving last delta CRL WAL entry")
}
}
resp := &logical.Response{
@ -413,6 +623,11 @@ func revokeCert(ctx context.Context, b *backend, req *logical.Request, serial st
}
func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew bool) error {
sc := b.makeStorageContext(ctx, req.Storage)
return buildAnyCRLs(sc, forceNew, false)
}
func buildAnyCRLs(sc *storageContext, forceNew bool, isDelta bool) error {
// In order to build all CRLs, we need knowledge of all issuers. Any two
// issuers with the same keys _and_ subject should have the same CRL since
// they're functionally equivalent.
@ -442,11 +657,10 @@ func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew b
var err error
var issuers []issuerID
var wasLegacy bool
sc := b.makeStorageContext(ctx, req.Storage)
// First, fetch an updated copy of the CRL config. We'll pass this into
// buildCRL.
globalCRLConfig, err := b.crlBuilder.getConfigWithUpdate(sc)
globalCRLConfig, err := sc.Backend.crlBuilder.getConfigWithUpdate(sc)
if err != nil {
return fmt.Errorf("error building CRL: while updating config: %v", err)
}
@ -463,7 +677,7 @@ func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew b
return nil
}
if !b.useLegacyBundleCaStorage() {
if !sc.Backend.useLegacyBundleCaStorage() {
issuers, err = sc.listIssuers()
if err != nil {
return fmt.Errorf("error building CRL: while listing issuers: %v", err)
@ -474,6 +688,14 @@ func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew b
// below for revocation to handle the legacy bundle.
issuers = []issuerID{legacyBundleShimID}
wasLegacy = true
// Here, we avoid building a delta CRL with the legacy CRL bundle.
//
// Users should upgrade symmetrically, rather than attempting
// backward compatibility for new features across disparate versions.
if isDelta {
return nil
}
}
config, err := sc.getIssuersConfig()
@ -539,11 +761,44 @@ func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew b
return fmt.Errorf("error building CRLs: unable to fetch cluster-local CRL configuration: %v", err)
}
// Before we load cert entries, we want to store the last seen delta WAL
// serial number. The subsequent List will have at LEAST that certificate
// (and potentially more) in it; when we're done writing the delta CRL,
// we'll write this serial as a sentinel to see if we need to rebuild it
// in the future.
var lastDeltaSerial string
if isDelta {
lastWALEntry, err := sc.Storage.Get(sc.Context, deltaWALLastRevokedSerial)
if err != nil {
return err
}
if lastWALEntry != nil && lastWALEntry.Value != nil {
var walInfo lastWALInfo
if err := lastWALEntry.DecodeJSON(&walInfo); err != nil {
return err
}
lastDeltaSerial = walInfo.Serial
}
}
// We fetch a list of delta WAL entries prior to generating the complete
// CRL. This allows us to avoid a lock (to clear such storage): anything
// visible now, should also be visible on the complete CRL we're writing.
var currDeltaCerts []string
if !isDelta {
currDeltaCerts, err = sc.Backend.crlBuilder.getPresentDeltaWALForClearing(sc)
if err != nil {
return fmt.Errorf("error building CRLs: unable to get present delta WAL entries for removal: %v", err)
}
}
// Next, we load and parse all revoked certificates. We need to assign
// these certificates to an issuer. Some certificates will not be
// assignable (if they were issued by a since-deleted issuer), so we need
// a separate pool for those.
unassignedCerts, revokedCertsMap, err := getRevokedCertEntries(ctx, req, issuerIDCertMap)
unassignedCerts, revokedCertsMap, err := getRevokedCertEntries(sc, issuerIDCertMap, isDelta)
if err != nil {
return fmt.Errorf("error building CRLs: unable to get revoked certificate entries: %v", err)
}
@ -632,13 +887,31 @@ func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew b
crlNumber := crlConfig.CRLNumberMap[crlIdentifier]
crlConfig.CRLNumberMap[crlIdentifier] += 1
// CRLs (regardless of complete vs delta) are incrementally
// numbered. But delta CRLs need to know the number of the
// last complete CRL. We assume that's the previous identifier
// if no value presently exists.
lastCompleteNumber, haveLast := crlConfig.LastCompleteNumberMap[crlIdentifier]
if !haveLast {
// We use the value of crlNumber for the current CRL, so
// decrement it by one to find the last one.
lastCompleteNumber = crlNumber - 1
}
// Lastly, build the CRL.
nextUpdate, err := buildCRL(sc, globalCRLConfig, forceNew, representative, revokedCerts, crlIdentifier, crlNumber)
nextUpdate, err := buildCRL(sc, globalCRLConfig, forceNew, representative, revokedCerts, crlIdentifier, crlNumber, isDelta, lastCompleteNumber)
if err != nil {
return fmt.Errorf("error building CRLs: unable to build CRL for issuer (%v): %v", representative, err)
}
crlConfig.CRLExpirationMap[crlIdentifier] = *nextUpdate
if !isDelta {
crlConfig.LastCompleteNumberMap[crlIdentifier] = crlNumber
} else if !haveLast {
// Since we're writing this config anyways, save our guess
// as to the last CRL number.
crlConfig.LastCompleteNumberMap[crlIdentifier] = lastCompleteNumber
}
}
}
@ -676,7 +949,7 @@ func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew b
}
if !stillHaveIssuerForID {
if err := req.Storage.Delete(ctx, "crls/"+crlId.String()); err != nil {
if err := sc.Storage.Delete(sc.Context, "crls/"+crlId.String()); err != nil {
return fmt.Errorf("error building CRLs: unable to clean up deleted issuers' CRL: %v", err)
}
}
@ -690,6 +963,37 @@ func buildCRLs(ctx context.Context, b *backend, req *logical.Request, forceNew b
}
}
if !isDelta {
// After we've confirmed the primary CRLs have built OK, go ahead and
// clear the delta CRL WAL and rebuild it.
if err := sc.Backend.crlBuilder.clearDeltaWAL(sc, currDeltaCerts); err != nil {
return fmt.Errorf("error building CRLs: unable to clear Delta WAL: %v", err)
}
if err := sc.Backend.crlBuilder.rebuildDeltaCRLsHoldingLock(sc, forceNew); err != nil {
return fmt.Errorf("error building CRLs: unable to rebuild empty Delta WAL: %v", err)
}
} else {
// Update our last build time here so we avoid checking for new certs
// for a while.
sc.Backend.crlBuilder.lastDeltaRebuildCheck = time.Now()
if len(lastDeltaSerial) > 0 {
// When we have a last delta serial, write out the relevant info
// so we can skip extra CRL rebuilds.
deltaInfo := lastDeltaInfo{Serial: lastDeltaSerial}
lastDeltaBuildEntry, err := logical.StorageEntryJSON(deltaWALLastBuildSerial, deltaInfo)
if err != nil {
return fmt.Errorf("error creating last delta CRL rebuild serial entry: %v", err)
}
err = sc.Storage.Put(sc.Context, lastDeltaBuildEntry)
if err != nil {
return fmt.Errorf("error persisting last delta CRL rebuild info: %v", err)
}
}
}
// All good :-)
return nil
}
@ -719,11 +1023,16 @@ func associateRevokedCertWithIsssuer(revInfo *revocationInfo, revokedCert *x509.
return false
}
func getRevokedCertEntries(ctx context.Context, req *logical.Request, issuerIDCertMap map[issuerID]*x509.Certificate) ([]pkix.RevokedCertificate, map[issuerID][]pkix.RevokedCertificate, error) {
func getRevokedCertEntries(sc *storageContext, issuerIDCertMap map[issuerID]*x509.Certificate, isDelta bool) ([]pkix.RevokedCertificate, map[issuerID][]pkix.RevokedCertificate, error) {
var unassignedCerts []pkix.RevokedCertificate
revokedCertsMap := make(map[issuerID][]pkix.RevokedCertificate)
revokedSerials, err := req.Storage.List(ctx, revokedPath)
listingPath := revokedPath
if isDelta {
listingPath = deltaWALPath
}
revokedSerials, err := sc.Storage.List(sc.Context, listingPath)
if err != nil {
return nil, nil, errutil.InternalError{Err: fmt.Sprintf("error fetching list of revoked certs: %s", err)}
}
@ -736,11 +1045,17 @@ func getRevokedCertEntries(ctx context.Context, req *logical.Request, issuerIDCe
}
for _, serial := range revokedSerials {
if isDelta && (serial == deltaWALLastBuildSerialName || serial == deltaWALLastRevokedSerialName) {
// Skip our placeholder entries...
continue
}
var revInfo revocationInfo
revokedEntry, err := req.Storage.Get(ctx, revokedPath+serial)
revokedEntry, err := sc.Storage.Get(sc.Context, revokedPath+serial)
if err != nil {
return nil, nil, errutil.InternalError{Err: fmt.Sprintf("unable to fetch revoked cert with serial %s: %s", serial, err)}
}
if revokedEntry == nil {
return nil, nil, errutil.InternalError{Err: fmt.Sprintf("revoked certificate entry for serial %s is nil", serial)}
}
@ -827,7 +1142,7 @@ func getRevokedCertEntries(ctx context.Context, req *logical.Request, issuerIDCe
return nil, nil, fmt.Errorf("error creating revocation entry for existing cert: %v", serial)
}
err = req.Storage.Put(ctx, revokedEntry)
err = sc.Storage.Put(sc.Context, revokedEntry)
if err != nil {
return nil, nil, fmt.Errorf("error updating revoked certificate at existing location: %v", serial)
}
@ -873,7 +1188,7 @@ func augmentWithRevokedIssuers(issuerIDEntryMap map[issuerID]*issuerEntry, issue
// Builds a CRL by going through the list of revoked certificates and building
// a new CRL with the stored revocation times and serial numbers.
func buildCRL(sc *storageContext, crlInfo *crlConfig, forceNew bool, thisIssuerId issuerID, revoked []pkix.RevokedCertificate, identifier crlID, crlNumber int64) (*time.Time, error) {
func buildCRL(sc *storageContext, crlInfo *crlConfig, forceNew bool, thisIssuerId issuerID, revoked []pkix.RevokedCertificate, identifier crlID, crlNumber int64, isDelta bool, lastCompleteNumber int64) (*time.Time, error) {
var revokedCerts []pkix.RevokedCertificate
crlLifetime, err := time.ParseDuration(crlInfo.Expiry)
@ -913,12 +1228,18 @@ WRITE:
now := time.Now()
nextUpdate := now.Add(crlLifetime)
ext, err := certutil.CreateDeltaCRLIndicatorExt(lastCompleteNumber)
if err != nil {
return nil, fmt.Errorf("could not create crl delta indicator extension: %v", err)
}
revocationListTemplate := &x509.RevocationList{
RevokedCertificates: revokedCerts,
Number: big.NewInt(crlNumber),
ThisUpdate: now,
NextUpdate: nextUpdate,
SignatureAlgorithm: signingBundle.RevocationSigAlg,
ExtraExtensions: []pkix.Extension{ext},
}
crlBytes, err := x509.CreateRevocationList(rand.Reader, revocationListTemplate, signingBundle.Certificate, signingBundle.PrivateKey)
@ -931,6 +1252,9 @@ WRITE:
// Ignore the CRL ID as it won't be persisted anyways; hard-code the
// old legacy path and allow it to be updated.
writePath = legacyCRLPath
} else if isDelta {
// Write the delta CRL to a unique storage location.
writePath += deltaCRLPathSuffix
}
err = sc.Storage.Put(sc.Context, &logical.StorageEntry{

View File

@ -21,6 +21,8 @@ type crlConfig struct {
AutoRebuild bool `json:"auto_rebuild"`
AutoRebuildGracePeriod string `json:"auto_rebuild_grace_period"`
OcspExpiry string `json:"ocsp_expiry"`
EnableDelta bool `json:"enable_delta"`
DeltaRebuildInterval string `json:"delta_rebuild_interval"`
}
// Implicit default values for the config if it does not exist.
@ -32,6 +34,8 @@ var defaultCrlConfig = crlConfig{
OcspExpiry: "12h",
AutoRebuild: false,
AutoRebuildGracePeriod: "12h",
EnableDelta: false,
DeltaRebuildInterval: "15m",
}
func pathConfigCRL(b *backend) *framework.Path {
@ -67,6 +71,15 @@ the NextUpdate field); defaults to 12 hours`,
Description: `The time before the CRL expires to automatically rebuild it, when enabled. Must be shorter than the CRL expiry. Defaults to 12h.`,
Default: "12h",
},
"enable_delta": {
Type: framework.TypeBool,
Description: `Whether to enable delta CRLs between authoritative CRL rebuilds`,
},
"delta_rebuild_interval": {
Type: framework.TypeString,
Description: `The time between delta CRL rebuilds if a new revocation has occurred. Must be shorter than the CRL expiry. Defaults to 15m.`,
Default: "15m",
},
},
Operations: map[logical.Operation]framework.OperationHandler{
@ -101,6 +114,8 @@ func (b *backend) pathCRLRead(ctx context.Context, req *logical.Request, _ *fram
"ocsp_expiry": config.OcspExpiry,
"auto_rebuild": config.AutoRebuild,
"auto_rebuild_grace_period": config.AutoRebuildGracePeriod,
"enable_delta": config.EnableDelta,
"delta_rebuild_interval": config.DeltaRebuildInterval,
},
}, nil
}
@ -155,15 +170,37 @@ func (b *backend) pathCRLWrite(ctx context.Context, req *logical.Request, d *fra
config.AutoRebuildGracePeriod = autoRebuildGracePeriod
}
if config.AutoRebuild {
expiry, _ := time.ParseDuration(config.Expiry)
gracePeriod, _ := time.ParseDuration(config.AutoRebuildGracePeriod)
if enableDeltaRaw, ok := d.GetOk("enable_delta"); ok {
config.EnableDelta = enableDeltaRaw.(bool)
}
if deltaRebuildIntervalRaw, ok := d.GetOk("delta_rebuild_interval"); ok {
deltaRebuildInterval := deltaRebuildIntervalRaw.(string)
if _, err := time.ParseDuration(deltaRebuildInterval); err != nil {
return logical.ErrorResponse(fmt.Sprintf("given delta_rebuild_interval could not be decoded: %s", err)), nil
}
config.DeltaRebuildInterval = deltaRebuildInterval
}
expiry, _ := time.ParseDuration(config.Expiry)
if config.AutoRebuild {
gracePeriod, _ := time.ParseDuration(config.AutoRebuildGracePeriod)
if gracePeriod >= expiry {
return logical.ErrorResponse(fmt.Sprintf("CRL auto-rebuilding grace period (%v) must be strictly shorter than CRL expiry (%v) value when auto-rebuilding of CRLs is enabled", config.AutoRebuildGracePeriod, config.Expiry)), nil
}
}
if config.EnableDelta {
deltaRebuildInterval, _ := time.ParseDuration(config.DeltaRebuildInterval)
if deltaRebuildInterval >= expiry {
return logical.ErrorResponse(fmt.Sprintf("CRL delta rebuild window (%v) must be strictly shorter than CRL expiry (%v) value when delta CRLs are enabled", config.DeltaRebuildInterval, config.Expiry)), nil
}
}
if config.EnableDelta && !config.AutoRebuild {
return logical.ErrorResponse(fmt.Sprintf("Delta CRLs cannot be enabled when auto rebuilding is disabled as the complete CRL is always regenerated!")), nil
}
entry, err := logical.StorageEntryJSON("config/crl", config)
if err != nil {
return nil, err

View File

@ -46,7 +46,7 @@ func pathFetchCAChain(b *backend) *framework.Path {
// Returns the CRL in raw format
func pathFetchCRL(b *backend) *framework.Path {
return &framework.Path{
Pattern: `crl(/pem)?`,
Pattern: `crl(/pem|/delta(/pem)?)?`,
Operations: map[logical.Operation]framework.OperationHandler{
logical.ReadOperation: &framework.PathOperation{
@ -109,7 +109,7 @@ hyphen-separated octal`,
// This returns the CRL in a non-raw format
func pathFetchCRLViaCertPath(b *backend) *framework.Path {
return &framework.Path{
Pattern: `cert/crl`,
Pattern: `cert/(crl|delta-crl)`,
Operations: map[logical.Operation]framework.OperationHandler{
logical.ReadOperation: &framework.PathOperation{
@ -178,15 +178,21 @@ func (b *backend) pathFetchRead(ctx context.Context, req *logical.Request, data
if req.Path == "ca_chain" {
contentType = "application/pkix-cert"
}
case req.Path == "crl" || req.Path == "crl/pem":
case req.Path == "crl" || req.Path == "crl/pem" || req.Path == "crl/delta" || req.Path == "crl/delta/pem":
serial = legacyCRLPath
if req.Path == "crl/delta" || req.Path == "crl/delta/pem" {
serial = deltaCRLPath
}
contentType = "application/pkix-crl"
if req.Path == "crl/pem" {
if req.Path == "crl/pem" || req.Path == "crl/delta/pem" {
pemType = "X509 CRL"
contentType = "application/x-pem-file"
}
case req.Path == "cert/crl":
case req.Path == "cert/crl" || req.Path == "cert/delta-crl":
serial = legacyCRLPath
if req.Path == "cert/delta-crl" {
serial = deltaCRLPath
}
pemType = "X509 CRL"
case strings.HasSuffix(req.Path, "/pem") || strings.HasSuffix(req.Path, "/raw"):
serial = data.Get("serial").(string)

View File

@ -875,7 +875,7 @@ the certificate.
)
func pathGetIssuerCRL(b *backend) *framework.Path {
pattern := "issuer/" + framework.GenericNameRegex(issuerRefParam) + "/crl(/pem|/der)?"
pattern := "issuer/" + framework.GenericNameRegex(issuerRefParam) + "/crl(/pem|/der|/delta(/pem|/der)?)?"
return buildPathGetIssuerCRL(b, pattern)
}
@ -919,6 +919,10 @@ func (b *backend) pathGetIssuerCRL(ctx context.Context, req *logical.Request, da
return nil, err
}
if strings.Contains(req.Path, "delta") {
crlPath += deltaCRLPathSuffix
}
crlEntry, err := req.Storage.Get(ctx, crlPath)
if err != nil {
return nil, err

View File

@ -25,6 +25,8 @@ const (
legacyMigrationBundleLogKey = "config/legacyMigrationBundleLog"
legacyCertBundlePath = "config/ca_bundle"
legacyCRLPath = "crl"
deltaCRLPath = "delta-crl"
deltaCRLPathSuffix = "-delta"
// Used as a quick sanity check for a reference id lookups...
uuidLength = 36
@ -159,9 +161,10 @@ type issuerEntry struct {
}
type localCRLConfigEntry struct {
IssuerIDCRLMap map[issuerID]crlID `json:"issuer_id_crl_map"`
CRLNumberMap map[crlID]int64 `json:"crl_number_map"`
CRLExpirationMap map[crlID]time.Time `json:"crl_expiration_map"`
IssuerIDCRLMap map[issuerID]crlID `json:"issuer_id_crl_map"`
CRLNumberMap map[crlID]int64 `json:"crl_number_map"`
LastCompleteNumberMap map[crlID]int64 `json:"last_complete_number_map"`
CRLExpirationMap map[crlID]time.Time `json:"crl_expiration_map"`
}
type keyConfigEntry struct {
@ -829,6 +832,24 @@ func (sc *storageContext) getLocalCRLConfig() (*localCRLConfigEntry, error) {
mapping.CRLNumberMap = make(map[crlID]int64)
}
if len(mapping.LastCompleteNumberMap) == 0 {
mapping.LastCompleteNumberMap = make(map[crlID]int64)
// Since this might not exist on migration, we want to guess as
// to the last full CRL number was. This was likely the last
// value from CRLNumberMap if it existed, since we're just adding
// the mapping here in this block.
//
// After the next full CRL build, we will have set this value
// correctly, so it doesn't really matter in the long term if
// we're off here.
for id, number := range mapping.CRLNumberMap {
// Decrement by one, since CRLNumberMap is the future number,
// not the last built number.
mapping.LastCompleteNumberMap[id] = number - 1
}
}
if len(mapping.CRLExpirationMap) == 0 {
mapping.CRLExpirationMap = make(map[crlID]time.Time)
}

3
changelog/16773.txt Normal file
View File

@ -0,0 +1,3 @@
```release-note:improvement
secrets/pki: Support generating delta CRLs for up-to-date CRLs when auto-building is enabled.
```

View File

@ -64,6 +64,11 @@ var SignatureAlgorithmNames = map[string]x509.SignatureAlgorithm{
"ed25519": x509.PureEd25519, // Duplicated for clarity; most won't expect the "Pure" prefix.
}
// OID for RFC 5280 Delta CRL Indicator CRL extension.
//
// > id-ce-deltaCRLIndicator OBJECT IDENTIFIER ::= { id-ce 27 }
var DeltaCRLIndicatorOID = asn1.ObjectIdentifier([]int{2, 5, 29, 27})
// GetHexFormatted returns the byte buffer formatted in hex with
// the specified separator between bytes.
func GetHexFormatted(buf []byte, sep string) string {
@ -1296,3 +1301,26 @@ func CreateKeyBundleWithKeyGenerator(keyType string, keyBits int, randReader io.
}
return result, nil
}
// CreateDeltaCRLIndicatorExt allows creating correctly formed delta CRLs
// that point back to the last complete CRL that they're based on.
func CreateDeltaCRLIndicatorExt(completeCRLNumber int64) (pkix.Extension, error) {
bigNum := big.NewInt(completeCRLNumber)
bigNumValue, err := asn1.Marshal(bigNum)
if err != nil {
return pkix.Extension{}, fmt.Errorf("unable to marshal complete CRL number (%v): %v", completeCRLNumber, err)
}
return pkix.Extension{
Id: DeltaCRLIndicatorOID,
// > When a conforming CRL issuer generates a delta CRL, the delta
// > CRL MUST include a critical delta CRL indicator extension.
Critical: true,
// This extension only includes the complete CRL number:
//
// > BaseCRLNumber ::= CRLNumber
//
// But, this needs to be encoded as a big number for encoding/asn1
// to work properly.
Value: bigNumValue,
}, nil
}

View File

@ -1104,18 +1104,35 @@ the newer `/pki/issuer/:issuer_ref/crl` path; the latter correctly places the
PEM-encoded CRL in the `crl` field whereas the former incorrectly places it
in the `certificate` field.
Endpoints with type `complete` are full CRLs containing all revoked
certificates (as of the time of generation. Endpoints with type `delta`
contain incremental CRLs on top of the last complete CRL, with any new
certificates that have been revoked. See the [revocation configuration
section](#set-crl-configuration) for more information about these options.
The delta CRL clears when the next complete CRL is rebuilt. Consumers of
delta CRLs will need to update their client to support fetching the
corresponding full CRL when it has been regenerated; otherwise, some serial
numbers may not appear in the local copy of the full CRL if the remote
complete and delta CRLs has been regenerated.
These are unauthenticated endpoints.
~> **Note**: As of Vault 1.11.0, these endpoints now serve a [version 2](https://datatracker.ietf.org/doc/html/rfc5280#section-5.1.2.1) CRL response.
| Method | Path | Issuer | Format |
| :----- | :-------------------------------- | :-------- |:----------------------------------------------------------------------------------|
| `GET` | `/pki/cert/crl` | `default` | JSON |
| `GET` | `/pki/crl` | `default` | DER [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") |
| `GET` | `/pki/crl/pem` | `default` | PEM [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") |
| `GET` | `/pki/issuer/:issuer_ref/crl` | Selected | JSON |
| `GET` | `/pki/issuer/:issuer_ref/crl/der` | Selected | DER [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") |
| `GET` | `/pki/issuer/:issuer_ref/crl/pem` | Selected | PEM [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") |
| Method | Path | Issuer | Format | Type |
| :----- | :-------------------------------------- | :-------- | :-------------------------------------------------------------------------------- | :------- |
| `GET` | `/pki/cert/crl` | `default` | JSON | Complete |
| `GET` | `/pki/crl` | `default` | DER [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Complete |
| `GET` | `/pki/crl/pem` | `default` | PEM [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Complete |
| `GET` | `/pki/cert/delta-crl` | `default` | JSON | Delta |
| `GET` | `/pki/crl/delta` | `default` | DER [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Delta |
| `GET` | `/pki/crl/delta/pem` | `default` | PEM [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Delta |
| `GET` | `/pki/issuer/:issuer_ref/crl` | Selected | JSON | Complete |
| `GET` | `/pki/issuer/:issuer_ref/crl/der` | Selected | DER [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Complete |
| `GET` | `/pki/issuer/:issuer_ref/crl/pem` | Selected | PEM [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Complete |
| `GET` | `/pki/issuer/:issuer_ref/crl/delta` | Selected | JSON | Delta |
| `GET` | `/pki/issuer/:issuer_ref/crl/delta/der` | Selected | DER [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Delta |
| `GET` | `/pki/issuer/:issuer_ref/crl/delta/pem` | Selected | PEM [\[1\]](#vault-cli-with-der-pem-responses "Vault CLI With DER/PEM Responses") | Delta |
#### Parameters
@ -2992,7 +3009,9 @@ $ curl \
"ocsp_disable": false,
"ocsp_expiry": "12h",
"auto_rebuild": false,
"auto_rebuild_grace_period": "12h"
"auto_rebuild_grace_period": "12h",
"enable_delta": false,
"delta_rebuild_interval": "15m"
},
"auth": null
}
@ -3025,6 +3044,10 @@ the CRL.
should not be relied upon for the latest certificate status information,
and OCSP should be used instead.
~> Note: The periodic function which controls automatic rebuilding of CRLs
and delta CRLs only executes once a minute; this prevents high system load
but limits the granularity of the temporal options below.
| Method | Path |
| :----- | :---------------- |
| `POST` | `/pki/config/crl` |
@ -3041,6 +3064,12 @@ the CRL.
the CRL upon expiry.
- `auto_rebuild_grace_period` `(string: "12h")` - Grace period before CRL expiry
to attempt rebuild of CRL. Must be shorter than the CRL expiry period.
- `enable_delta` `(bool: false)` - Enables or disables building of delta CRLs
with up-to-date revocation information, augmenting the last complete CRL.
This option requires `auto_rebuild` to also be enabled.
- `delta_rebuild_interval` `(string: "15m")` - Interval to check for new
revocations on, to regenerate the delta CRL. Must be shorter than CRL
expiry.
#### Sample Payload
@ -3051,7 +3080,9 @@ the CRL.
"ocsp_disable": "false",
"ocsp_expiry": "12h",
"auto_rebuild": "true",
"auto_rebuild_grace_period": "8h"
"auto_rebuild_grace_period": "8h",
"enable_delta": "true",
"delta_rebuild_interval": "10m",
}
```