Add support for batch key rotation (#16844)

This commit is contained in:
Poorna 2023-04-04 10:56:54 -07:00 committed by GitHub
parent 51f7f9aaa3
commit 3158f2d12e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 3724 additions and 64 deletions

View File

@ -249,6 +249,7 @@ type BatchJobRequest struct {
Started time.Time `yaml:"-" json:"started"`
Location string `yaml:"-" json:"location"`
Replicate *BatchJobReplicateV1 `yaml:"replicate" json:"replicate"`
KeyRotate *BatchJobKeyRotateV1 `yaml:"keyrotate" json:"keyrotate"`
ctx context.Context `msg:"-"`
}
@ -570,7 +571,7 @@ func (r *BatchJobReplicateV1) StartFromSource(ctx context.Context, api ObjectLay
ri.trackCurrentBucketObject(r.Target.Bucket, oi, success)
globalBatchJobsMetrics.save(job.ID, ri)
// persist in-memory state to disk after every 10secs.
logger.LogIf(ctx, ri.updateAfter(ctx, api, 10*time.Second, job.Location))
logger.LogIf(ctx, ri.updateAfter(ctx, api, 10*time.Second, job))
}()
}
wk.Wait()
@ -581,7 +582,7 @@ func (r *BatchJobReplicateV1) StartFromSource(ctx context.Context, api ObjectLay
globalBatchJobsMetrics.save(job.ID, ri)
// persist in-memory state to disk.
logger.LogIf(ctx, ri.updateAfter(ctx, api, 0, job.Location))
logger.LogIf(ctx, ri.updateAfter(ctx, api, 0, job))
buf, _ := json.Marshal(ri)
if err := r.Notify(ctx, bytes.NewReader(buf)); err != nil {
@ -765,14 +766,34 @@ const (
)
func (ri *batchJobInfo) load(ctx context.Context, api ObjectLayer, job BatchJobRequest) error {
data, err := readConfig(ctx, api, pathJoin(job.Location, batchReplName))
var fileName string
var format, version uint16
switch {
case job.Replicate != nil:
fileName = batchReplName
version = batchReplVersionV1
format = batchReplFormat
case job.KeyRotate != nil:
fileName = batchKeyRotationName
version = batchKeyRotateVersionV1
format = batchKeyRotationFormat
}
data, err := readConfig(ctx, api, pathJoin(job.Location, fileName))
if err != nil {
if errors.Is(err, errConfigNotFound) || isErrObjectNotFound(err) {
ri.Version = batchReplVersionV1
if job.Replicate.Flags.Retry.Attempts > 0 {
ri.RetryAttempts = job.Replicate.Flags.Retry.Attempts
} else {
ri.Version = int(version)
switch {
case job.Replicate != nil:
ri.RetryAttempts = batchReplJobDefaultRetries
if job.Replicate.Flags.Retry.Attempts > 0 {
ri.RetryAttempts = job.Replicate.Flags.Retry.Attempts
}
case job.KeyRotate != nil:
ri.RetryAttempts = batchKeyRotateJobDefaultRetries
if job.KeyRotate.Flags.Retry.Attempts > 0 {
ri.RetryAttempts = job.KeyRotate.Flags.Retry.Attempts
}
}
return nil
}
@ -783,18 +804,18 @@ func (ri *batchJobInfo) load(ctx context.Context, api ObjectLayer, job BatchJobR
return nil
}
if len(data) <= 4 {
return fmt.Errorf("batchRepl: no data")
return fmt.Errorf("%s: no data", ri.JobType)
}
// Read header
switch binary.LittleEndian.Uint16(data[0:2]) {
case batchReplFormat:
case format:
default:
return fmt.Errorf("batchRepl: unknown format: %d", binary.LittleEndian.Uint16(data[0:2]))
return fmt.Errorf("%s: unknown format: %d", ri.JobType, binary.LittleEndian.Uint16(data[0:2]))
}
switch binary.LittleEndian.Uint16(data[2:4]) {
case batchReplVersion:
case version:
default:
return fmt.Errorf("batchRepl: unknown version: %d", binary.LittleEndian.Uint16(data[2:4]))
return fmt.Errorf("%s: unknown version: %d", ri.JobType, binary.LittleEndian.Uint16(data[2:4]))
}
ri.mu.Lock()
@ -808,7 +829,7 @@ func (ri *batchJobInfo) load(ctx context.Context, api ObjectLayer, job BatchJobR
switch ri.Version {
case batchReplVersionV1:
default:
return fmt.Errorf("unexpected batch repl meta version: %d", ri.Version)
return fmt.Errorf("unexpected batch %s meta version: %d", ri.JobType, ri.Version)
}
return nil
@ -857,31 +878,51 @@ func (ri *batchJobInfo) countItem(size int64, dmarker, success bool) {
}
}
func (ri *batchJobInfo) updateAfter(ctx context.Context, api ObjectLayer, duration time.Duration, jobLocation string) error {
func (ri *batchJobInfo) updateAfter(ctx context.Context, api ObjectLayer, duration time.Duration, job BatchJobRequest) error {
if ri == nil {
return errInvalidArgument
}
now := UTCNow()
ri.mu.Lock()
var (
format, version uint16
jobTyp, fileName string
)
if now.Sub(ri.LastUpdate) >= duration {
switch job.Type() {
case madmin.BatchJobReplicate:
format = batchReplFormat
version = batchReplVersion
jobTyp = string(job.Type())
fileName = batchReplName
ri.Version = batchReplVersionV1
case madmin.BatchJobKeyRotate:
format = batchKeyRotationFormat
version = batchKeyRotateVersion
jobTyp = string(job.Type())
fileName = batchKeyRotationName
ri.Version = batchKeyRotateVersionV1
default:
return errInvalidArgument
}
if serverDebugLog {
console.Debugf("batchReplicate: persisting batchReplication info on drive: threshold:%s, batchRepl:%#v\n", now.Sub(ri.LastUpdate), ri)
console.Debugf("%s: persisting info on drive: threshold:%s, %s:%#v\n", jobTyp, now.Sub(ri.LastUpdate), jobTyp, ri)
}
ri.LastUpdate = now
ri.Version = batchReplVersionV1
data := make([]byte, 4, ri.Msgsize()+4)
// Initialize the header.
binary.LittleEndian.PutUint16(data[0:2], batchReplFormat)
binary.LittleEndian.PutUint16(data[2:4], batchReplVersion)
binary.LittleEndian.PutUint16(data[0:2], format)
binary.LittleEndian.PutUint16(data[2:4], version)
buf, err := ri.MarshalMsg(data)
ri.mu.Unlock()
if err != nil {
return err
}
return saveConfig(ctx, api, pathJoin(jobLocation, batchReplName), buf)
return saveConfig(ctx, api, pathJoin(job.Location, fileName), buf)
}
ri.mu.Unlock()
return nil
@ -1055,7 +1096,7 @@ func (r *BatchJobReplicateV1) Start(ctx context.Context, api ObjectLayer, job Ba
ri.trackCurrentBucketObject(r.Source.Bucket, result, success)
globalBatchJobsMetrics.save(job.ID, ri)
// persist in-memory state to disk after every 10secs.
logger.LogIf(ctx, ri.updateAfter(ctx, api, 10*time.Second, job.Location))
logger.LogIf(ctx, ri.updateAfter(ctx, api, 10*time.Second, job))
}()
}
wk.Wait()
@ -1066,7 +1107,7 @@ func (r *BatchJobReplicateV1) Start(ctx context.Context, api ObjectLayer, job Ba
globalBatchJobsMetrics.save(job.ID, ri)
// persist in-memory state to disk.
logger.LogIf(ctx, ri.updateAfter(ctx, api, 0, job.Location))
logger.LogIf(ctx, ri.updateAfter(ctx, api, 0, job))
buf, _ := json.Marshal(ri)
if err := r.Notify(ctx, bytes.NewReader(buf)); err != nil {
@ -1240,8 +1281,11 @@ func (r *BatchJobReplicateV1) Validate(ctx context.Context, job BatchJobRequest,
// Type returns type of batch job, currently only supports 'replicate'
func (j BatchJobRequest) Type() madmin.BatchJobType {
if j.Replicate != nil {
switch {
case j.Replicate != nil:
return madmin.BatchJobReplicate
case j.KeyRotate != nil:
return madmin.BatchJobKeyRotate
}
return madmin.BatchJobType("unknown")
}
@ -1249,20 +1293,28 @@ func (j BatchJobRequest) Type() madmin.BatchJobType {
// Validate validates the current job, used by 'save()' before
// persisting the job request
func (j BatchJobRequest) Validate(ctx context.Context, o ObjectLayer) error {
if j.Replicate != nil {
switch {
case j.Replicate != nil:
return j.Replicate.Validate(ctx, j, o)
case j.KeyRotate != nil:
return j.KeyRotate.Validate(ctx, j, o)
}
return errInvalidArgument
}
func (j BatchJobRequest) delete(ctx context.Context, api ObjectLayer) {
deleteConfig(ctx, api, pathJoin(j.Location, batchReplName))
switch {
case j.Replicate != nil:
deleteConfig(ctx, api, pathJoin(j.Location, batchReplName))
case j.KeyRotate != nil:
deleteConfig(ctx, api, pathJoin(j.Location, batchKeyRotationName))
}
globalBatchJobsMetrics.delete(j.ID)
deleteConfig(ctx, api, j.Location)
}
func (j *BatchJobRequest) save(ctx context.Context, api ObjectLayer) error {
if j.Replicate == nil {
if j.Replicate == nil && j.KeyRotate == nil {
return errInvalidArgument
}
@ -1524,6 +1576,10 @@ func (j *BatchJobPool) resume() {
return
}
for result := range results {
// ignore batch-replicate.bin and batch-rotate.bin entries
if strings.HasSuffix(result.Name, slashSeparator) {
continue
}
req := &BatchJobRequest{}
if err := req.load(ctx, j.objLayer, result.Name); err != nil {
logger.LogIf(ctx, err)
@ -1570,6 +1626,14 @@ func (j *BatchJobPool) AddWorker() {
}
}
}
if job.KeyRotate != nil {
if err := job.KeyRotate.Start(job.ctx, j.objLayer, *job); err != nil {
if !isErrBucketNotFound(err) {
logger.LogIf(j.ctx, err)
continue
}
}
}
job.delete(j.ctx, j.objLayer)
j.canceler(job.ID, false)
case <-j.workerKillCh:
@ -1645,25 +1709,32 @@ var globalBatchJobsMetrics = batchJobMetrics{
metrics: make(map[string]*batchJobInfo),
}
//msgp:ignore batchReplicationMetric
//go:generate stringer -type=batchReplicationMetric -trimprefix=batchReplicationMetric $GOFILE
type batchReplicationMetric uint8
//msgp:ignore batchJobMetric
//go:generate stringer -type=batchJobMetric -trimprefix=batchJobMetric $GOFILE
type batchJobMetric uint8
const (
batchReplicationMetricObject batchReplicationMetric = iota
batchReplicationMetricObject batchJobMetric = iota
batchKeyRotationMetricObject
)
func batchReplicationTrace(d batchReplicationMetric, job string, startTime time.Time, duration time.Duration, info ObjectInfo, attempts int, err error) madmin.TraceInfo {
func batchJobTrace(d batchJobMetric, job string, startTime time.Time, duration time.Duration, info ObjectInfo, attempts int, err error) madmin.TraceInfo {
var errStr string
if err != nil {
errStr = err.Error()
}
funcName := fmt.Sprintf("batchReplication.%s (job-name=%s)", d.String(), job)
jobKind := "batchReplication"
traceType := madmin.TraceBatchReplication
if d == batchKeyRotationMetricObject {
jobKind = "batchKeyRotation"
traceType = madmin.TraceBatchKeyRotation
}
funcName := fmt.Sprintf("%s.%s (job-name=%s)", jobKind, d.String(), job)
if attempts > 0 {
funcName = fmt.Sprintf("batchReplication.%s (job-name=%s,attempts=%s)", d.String(), job, humanize.Ordinal(attempts))
funcName = fmt.Sprintf("%s.%s (job-name=%s,attempts=%s)", jobKind, d.String(), job, humanize.Ordinal(attempts))
}
return madmin.TraceInfo{
TraceType: madmin.TraceBatchReplication,
TraceType: traceType,
Time: startTime,
NodeName: globalLocalNodeName,
FuncName: funcName,
@ -1695,6 +1766,12 @@ func (m *batchJobMetrics) report(jobID string) (metrics *madmin.BatchJobMetrics)
BytesTransferred: job.BytesTransferred,
BytesFailed: job.BytesFailed,
},
KeyRotate: &madmin.KeyRotationInfo{
Bucket: job.Bucket,
Object: job.Object,
Objects: job.Objects,
ObjectsFailed: job.ObjectsFailed,
},
}
if match {
break
@ -1717,12 +1794,19 @@ func (m *batchJobMetrics) save(jobID string, ri *batchJobInfo) {
m.metrics[jobID] = ri.clone()
}
func (m *batchJobMetrics) trace(d batchReplicationMetric, job string, attempts int, info ObjectInfo) func(err error) {
func (m *batchJobMetrics) trace(d batchJobMetric, job string, attempts int, info ObjectInfo) func(err error) {
startTime := time.Now()
return func(err error) {
duration := time.Since(startTime)
if globalTrace.NumSubscribers(madmin.TraceBatchReplication) > 0 {
globalTrace.Publish(batchReplicationTrace(d, job, startTime, duration, info, attempts, err))
switch d {
case batchReplicationMetricObject:
if globalTrace.NumSubscribers(madmin.TraceBatchReplication) > 0 {
globalTrace.Publish(batchJobTrace(d, job, startTime, duration, info, attempts, err))
}
case batchKeyRotationMetricObject:
if globalTrace.NumSubscribers(madmin.TraceBatchKeyRotation) > 0 {
globalTrace.Publish(batchJobTrace(d, job, startTime, duration, info, attempts, err))
}
}
}
}

View File

@ -1538,6 +1538,24 @@ func (z *BatchJobRequest) DecodeMsg(dc *msgp.Reader) (err error) {
return
}
}
case "KeyRotate":
if dc.IsNil() {
err = dc.ReadNil()
if err != nil {
err = msgp.WrapError(err, "KeyRotate")
return
}
z.KeyRotate = nil
} else {
if z.KeyRotate == nil {
z.KeyRotate = new(BatchJobKeyRotateV1)
}
err = z.KeyRotate.DecodeMsg(dc)
if err != nil {
err = msgp.WrapError(err, "KeyRotate")
return
}
}
default:
err = dc.Skip()
if err != nil {
@ -1551,9 +1569,9 @@ func (z *BatchJobRequest) DecodeMsg(dc *msgp.Reader) (err error) {
// EncodeMsg implements msgp.Encodable
func (z *BatchJobRequest) EncodeMsg(en *msgp.Writer) (err error) {
// map header, size 5
// map header, size 6
// write "ID"
err = en.Append(0x85, 0xa2, 0x49, 0x44)
err = en.Append(0x86, 0xa2, 0x49, 0x44)
if err != nil {
return
}
@ -1609,15 +1627,32 @@ func (z *BatchJobRequest) EncodeMsg(en *msgp.Writer) (err error) {
return
}
}
// write "KeyRotate"
err = en.Append(0xa9, 0x4b, 0x65, 0x79, 0x52, 0x6f, 0x74, 0x61, 0x74, 0x65)
if err != nil {
return
}
if z.KeyRotate == nil {
err = en.WriteNil()
if err != nil {
return
}
} else {
err = z.KeyRotate.EncodeMsg(en)
if err != nil {
err = msgp.WrapError(err, "KeyRotate")
return
}
}
return
}
// MarshalMsg implements msgp.Marshaler
func (z *BatchJobRequest) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
// map header, size 5
// map header, size 6
// string "ID"
o = append(o, 0x85, 0xa2, 0x49, 0x44)
o = append(o, 0x86, 0xa2, 0x49, 0x44)
o = msgp.AppendString(o, z.ID)
// string "User"
o = append(o, 0xa4, 0x55, 0x73, 0x65, 0x72)
@ -1639,6 +1674,17 @@ func (z *BatchJobRequest) MarshalMsg(b []byte) (o []byte, err error) {
return
}
}
// string "KeyRotate"
o = append(o, 0xa9, 0x4b, 0x65, 0x79, 0x52, 0x6f, 0x74, 0x61, 0x74, 0x65)
if z.KeyRotate == nil {
o = msgp.AppendNil(o)
} else {
o, err = z.KeyRotate.MarshalMsg(o)
if err != nil {
err = msgp.WrapError(err, "KeyRotate")
return
}
}
return
}
@ -1701,6 +1747,23 @@ func (z *BatchJobRequest) UnmarshalMsg(bts []byte) (o []byte, err error) {
return
}
}
case "KeyRotate":
if msgp.IsNil(bts) {
bts, err = msgp.ReadNilBytes(bts)
if err != nil {
return
}
z.KeyRotate = nil
} else {
if z.KeyRotate == nil {
z.KeyRotate = new(BatchJobKeyRotateV1)
}
bts, err = z.KeyRotate.UnmarshalMsg(bts)
if err != nil {
err = msgp.WrapError(err, "KeyRotate")
return
}
}
default:
bts, err = msgp.Skip(bts)
if err != nil {
@ -1721,6 +1784,12 @@ func (z *BatchJobRequest) Msgsize() (s int) {
} else {
s += z.Replicate.Msgsize()
}
s += 10
if z.KeyRotate == nil {
s += msgp.NilSize
} else {
s += z.KeyRotate.Msgsize()
}
return
}

564
cmd/batch-rotate.go Normal file
View File

@ -0,0 +1,564 @@
// Copyright (c) 2015-2023 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package cmd
import (
"bytes"
"context"
"encoding/base64"
"encoding/json"
"errors"
"fmt"
"io"
"math/rand"
"net/http"
"runtime"
"strconv"
"strings"
"time"
jsoniter "github.com/json-iterator/go"
"github.com/minio/minio-go/v7/pkg/tags"
"github.com/minio/minio/internal/crypto"
xhttp "github.com/minio/minio/internal/http"
"github.com/minio/minio/internal/kms"
"github.com/minio/minio/internal/logger"
"github.com/minio/minio/internal/workers"
"github.com/minio/pkg/env"
"github.com/minio/pkg/wildcard"
)
// keyrotate:
// apiVersion: v1
// bucket: BUCKET
// prefix: PREFIX
// encryption:
// type: sse-s3 # valid values are sse-s3 and sse-kms
// key: <new-kms-key> # valid only for sse-kms
// context: <new-kms-key-context> # valid only for sse-kms
// # optional flags based filtering criteria
// # for all objects
// flags:
// filter:
// newerThan: "7d" # match objects newer than this value (e.g. 7d10h31s)
// olderThan: "7d" # match objects older than this value (e.g. 7d10h31s)
// createdAfter: "date" # match objects created after "date"
// createdBefore: "date" # match objects created before "date"
// tags:
// - key: "name"
// value: "pick*" # match objects with tag 'name', with all values starting with 'pick'
// metadata:
// - key: "content-type"
// value: "image/*" # match objects with 'content-type', with all values starting with 'image/'
// kmskey: "key-id" # match objects with KMS key-id (applicable only for sse-kms)
// notify:
// endpoint: "https://notify.endpoint" # notification endpoint to receive job status events
// token: "Bearer xxxxx" # optional authentication token for the notification endpoint
// retry:
// attempts: 10 # number of retries for the job before giving up
// delay: "500ms" # least amount of delay between each retry
//go:generate msgp -file $GOFILE -unexported
// BatchKeyRotateKV is a datatype that holds key and values for filtering of objects
// used by metadata filter as well as tags based filtering.
type BatchKeyRotateKV struct {
Key string `yaml:"key" json:"key"`
Value string `yaml:"value" json:"value"`
}
// Validate returns an error if key is empty
func (kv BatchKeyRotateKV) Validate() error {
if kv.Key == "" {
return errInvalidArgument
}
return nil
}
// Empty indicates if kv is not set
func (kv BatchKeyRotateKV) Empty() bool {
return kv.Key == "" && kv.Value == ""
}
// Match matches input kv with kv, value will be wildcard matched depending on the user input
func (kv BatchKeyRotateKV) Match(ikv BatchKeyRotateKV) bool {
if kv.Empty() {
return true
}
if strings.EqualFold(kv.Key, ikv.Key) {
return wildcard.Match(kv.Value, ikv.Value)
}
return false
}
// BatchKeyRotateRetry datatype represents total retry attempts and delay between each retries.
type BatchKeyRotateRetry struct {
Attempts int `yaml:"attempts" json:"attempts"` // number of retry attempts
Delay time.Duration `yaml:"delay" json:"delay"` // delay between each retries
}
// Validate validates input replicate retries.
func (r BatchKeyRotateRetry) Validate() error {
if r.Attempts < 0 {
return errInvalidArgument
}
if r.Delay < 0 {
return errInvalidArgument
}
return nil
}
// BatchKeyRotationType defines key rotation type
type BatchKeyRotationType string
const (
sses3 BatchKeyRotationType = "sse-s3"
ssekms BatchKeyRotationType = "sse-kms"
)
// BatchJobKeyRotateEncryption defines key rotation encryption options passed
type BatchJobKeyRotateEncryption struct {
Type BatchKeyRotationType `yaml:"type" json:"type"`
Key string `yaml:"key" json:"key"`
Context string `yaml:"context" json:"context"`
kmsContext kms.Context `msg:"-"`
}
// Validate validates input key rotation encryption options.
func (e BatchJobKeyRotateEncryption) Validate() error {
if e.Type != sses3 && e.Type != ssekms {
return errInvalidArgument
}
spaces := strings.HasPrefix(e.Key, " ") || strings.HasSuffix(e.Key, " ")
if e.Type == ssekms && spaces {
return crypto.ErrInvalidEncryptionKeyID
}
if e.Type == ssekms && GlobalKMS != nil {
ctx := kms.Context{}
if e.Context != "" {
b, err := base64.StdEncoding.DecodeString(e.Context)
if err != nil {
return err
}
json := jsoniter.ConfigCompatibleWithStandardLibrary
if err := json.Unmarshal(b, &ctx); err != nil {
return err
}
}
e.kmsContext = kms.Context{}
for k, v := range ctx {
e.kmsContext[k] = v
}
ctx["MinIO batch API"] = "batchrotate" // Context for a test key operation
if _, err := GlobalKMS.GenerateKey(GlobalContext, e.Key, ctx); err != nil {
return err
}
}
return nil
}
// BatchKeyRotateFilter holds all the filters currently supported for batch replication
type BatchKeyRotateFilter struct {
NewerThan time.Duration `yaml:"newerThan,omitempty" json:"newerThan"`
OlderThan time.Duration `yaml:"olderThan,omitempty" json:"olderThan"`
CreatedAfter time.Time `yaml:"createdAfter,omitempty" json:"createdAfter"`
CreatedBefore time.Time `yaml:"createdBefore,omitempty" json:"createdBefore"`
Tags []BatchKeyRotateKV `yaml:"tags,omitempty" json:"tags"`
Metadata []BatchKeyRotateKV `yaml:"metadata,omitempty" json:"metadata"`
KMSKeyID string `yaml:"kmskeyid" json:"kmskey"`
}
// BatchKeyRotateNotification success or failure notification endpoint for each job attempts
type BatchKeyRotateNotification struct {
Endpoint string `yaml:"endpoint" json:"endpoint"`
Token string `yaml:"token" json:"token"`
}
// BatchJobKeyRotateFlags various configurations for replication job definition currently includes
// - filter
// - notify
// - retry
type BatchJobKeyRotateFlags struct {
Filter BatchKeyRotateFilter `yaml:"filter" json:"filter"`
Notify BatchKeyRotateNotification `yaml:"notify" json:"notify"`
Retry BatchKeyRotateRetry `yaml:"retry" json:"retry"`
}
// BatchJobKeyRotateV1 v1 of batch key rotation job
type BatchJobKeyRotateV1 struct {
APIVersion string `yaml:"apiVersion" json:"apiVersion"`
Flags BatchJobKeyRotateFlags `yaml:"flags" json:"flags"`
Bucket string `yaml:"bucket" json:"bucket"`
Prefix string `yaml:"prefix" json:"prefix"`
Endpoint string `yaml:"endpoint" json:"endpoint"`
Encryption BatchJobKeyRotateEncryption `yaml:"encryption" json:"encryption"`
}
// Notify notifies notification endpoint if configured regarding job failure or success.
func (r BatchJobKeyRotateV1) Notify(ctx context.Context, body io.Reader) error {
if r.Flags.Notify.Endpoint == "" {
return nil
}
ctx, cancel := context.WithTimeout(ctx, 10*time.Second)
defer cancel()
req, err := http.NewRequestWithContext(ctx, http.MethodPost, r.Flags.Notify.Endpoint, body)
if err != nil {
return err
}
if r.Flags.Notify.Token != "" {
req.Header.Set("Authorization", r.Flags.Notify.Token)
}
clnt := http.Client{Transport: getRemoteInstanceTransport}
resp, err := clnt.Do(req)
if err != nil {
return err
}
xhttp.DrainBody(resp.Body)
if resp.StatusCode != http.StatusOK {
return errors.New(resp.Status)
}
return nil
}
// KeyRotate rotates encryption key of an object
func (r *BatchJobKeyRotateV1) KeyRotate(ctx context.Context, api ObjectLayer, objInfo ObjectInfo) error {
srcBucket := r.Bucket
srcObject := objInfo.Name
if objInfo.DeleteMarker || !objInfo.VersionPurgeStatus.Empty() {
return nil
}
sseKMS := crypto.S3KMS.IsEncrypted(objInfo.UserDefined)
sseS3 := crypto.S3.IsEncrypted(objInfo.UserDefined)
if !sseKMS && !sseS3 { // neither sse-s3 nor sse-kms disallowed
return errInvalidEncryptionParameters
}
if sseKMS && r.Encryption.Type == sses3 { // previously encrypted with sse-kms, now sse-s3 disallowed
return errInvalidEncryptionParameters
}
versioned := globalBucketVersioningSys.PrefixEnabled(srcBucket, srcObject)
versionSuspended := globalBucketVersioningSys.PrefixSuspended(srcBucket, srcObject)
lock := api.NewNSLock(r.Bucket, objInfo.Name)
lkctx, err := lock.GetLock(ctx, globalOperationTimeout)
if err != nil {
return err
}
ctx = lkctx.Context()
defer lock.Unlock(lkctx)
opts := ObjectOptions{
VersionID: objInfo.VersionID,
Versioned: versioned,
VersionSuspended: versionSuspended,
NoLock: true,
}
obj, err := api.GetObjectInfo(ctx, r.Bucket, objInfo.Name, opts)
if err != nil {
return err
}
oi := obj.Clone()
var (
newKeyID string
newKeyContext kms.Context
)
encMetadata := make(map[string]string)
for k, v := range oi.UserDefined {
if strings.HasPrefix(strings.ToLower(k), ReservedMetadataPrefixLower) {
encMetadata[k] = v
}
}
if (sseKMS || sseS3) && r.Encryption.Type == ssekms {
if err = r.Encryption.Validate(); err != nil {
return err
}
newKeyID = strings.TrimPrefix(r.Encryption.Key, crypto.ARNPrefix)
newKeyContext = r.Encryption.kmsContext
}
if err = rotateKey(ctx, []byte{}, newKeyID, []byte{}, r.Bucket, oi.Name, encMetadata, newKeyContext); err != nil {
return err
}
// Since we are rotating the keys, make sure to update the metadata.
oi.metadataOnly = true
oi.keyRotation = true
for k, v := range encMetadata {
oi.UserDefined[k] = v
}
if _, err := api.CopyObject(ctx, r.Bucket, oi.Name, r.Bucket, oi.Name, oi, ObjectOptions{
VersionID: oi.VersionID,
}, ObjectOptions{
VersionID: oi.VersionID,
NoLock: true,
}); err != nil {
return err
}
return nil
}
const (
batchKeyRotationName = "batch-rotate.bin"
batchKeyRotationFormat = 1
batchKeyRotateVersionV1 = 1
batchKeyRotateVersion = batchKeyRotateVersionV1
batchKeyRotateAPIVersion = "v1"
batchKeyRotateJobDefaultRetries = 3
batchKeyRotateJobDefaultRetryDelay = 250 * time.Millisecond
)
// Start the batch key rottion job, resumes if there was a pending job via "job.ID"
func (r *BatchJobKeyRotateV1) Start(ctx context.Context, api ObjectLayer, job BatchJobRequest) error {
ri := &batchJobInfo{
JobID: job.ID,
JobType: string(job.Type()),
StartTime: job.Started,
}
if err := ri.load(ctx, api, job); err != nil {
return err
}
globalBatchJobsMetrics.save(job.ID, ri)
lastObject := ri.Object
delay := job.KeyRotate.Flags.Retry.Delay
if delay == 0 {
delay = batchKeyRotateJobDefaultRetryDelay
}
rnd := rand.New(rand.NewSource(time.Now().UnixNano()))
skip := func(info FileInfo) (ok bool) {
if r.Flags.Filter.OlderThan > 0 && time.Since(info.ModTime) < r.Flags.Filter.OlderThan {
// skip all objects that are newer than specified older duration
return false
}
if r.Flags.Filter.NewerThan > 0 && time.Since(info.ModTime) >= r.Flags.Filter.NewerThan {
// skip all objects that are older than specified newer duration
return false
}
if !r.Flags.Filter.CreatedAfter.IsZero() && r.Flags.Filter.CreatedAfter.Before(info.ModTime) {
// skip all objects that are created before the specified time.
return false
}
if !r.Flags.Filter.CreatedBefore.IsZero() && r.Flags.Filter.CreatedBefore.After(info.ModTime) {
// skip all objects that are created after the specified time.
return false
}
if len(r.Flags.Filter.Tags) > 0 {
// Only parse object tags if tags filter is specified.
tagMap := map[string]string{}
tagStr := info.Metadata[xhttp.AmzObjectTagging]
if len(tagStr) != 0 {
t, err := tags.ParseObjectTags(tagStr)
if err != nil {
return false
}
tagMap = t.ToMap()
}
for _, kv := range r.Flags.Filter.Tags {
for t, v := range tagMap {
if kv.Match(BatchKeyRotateKV{Key: t, Value: v}) {
return true
}
}
}
// None of the provided tags filter match skip the object
return false
}
if len(r.Flags.Filter.Metadata) > 0 {
for _, kv := range r.Flags.Filter.Metadata {
for k, v := range info.Metadata {
if !strings.HasPrefix(strings.ToLower(k), "x-amz-meta-") && !isStandardHeader(k) {
continue
}
// We only need to match x-amz-meta or standardHeaders
if kv.Match(BatchKeyRotateKV{Key: k, Value: v}) {
return true
}
}
}
// None of the provided metadata filters match skip the object.
return false
}
if r.Flags.Filter.KMSKeyID != "" {
if v, ok := info.Metadata[xhttp.AmzServerSideEncryptionKmsID]; ok && strings.TrimPrefix(v, crypto.ARNPrefix) != r.Flags.Filter.KMSKeyID {
return false
}
}
return true
}
workerSize, err := strconv.Atoi(env.Get("_MINIO_BATCH_KEYROTATION_WORKERS", strconv.Itoa(runtime.GOMAXPROCS(0)/2)))
if err != nil {
return err
}
wk, err := workers.New(workerSize)
if err != nil {
// invalid worker size.
return err
}
retryAttempts := ri.RetryAttempts
ctx, cancel := context.WithCancel(ctx)
results := make(chan ObjectInfo, 100)
if err := api.Walk(ctx, r.Bucket, r.Prefix, results, ObjectOptions{
WalkMarker: lastObject,
WalkFilter: skip,
}); err != nil {
cancel()
// Do not need to retry if we can't list objects on source.
return err
}
for result := range results {
result := result
sseKMS := crypto.S3KMS.IsEncrypted(result.UserDefined)
sseS3 := crypto.S3.IsEncrypted(result.UserDefined)
if !sseKMS && !sseS3 { // neither sse-s3 nor sse-kms disallowed
continue
}
wk.Take()
go func() {
defer wk.Give()
for attempts := 1; attempts <= retryAttempts; attempts++ {
attempts := attempts
stopFn := globalBatchJobsMetrics.trace(batchKeyRotationMetricObject, job.ID, attempts, result)
success := true
if err := r.KeyRotate(ctx, api, result); err != nil {
stopFn(err)
logger.LogIf(ctx, err)
success = false
} else {
stopFn(nil)
}
ri.trackCurrentBucketObject(r.Bucket, result, success)
ri.RetryAttempts = attempts
globalBatchJobsMetrics.save(job.ID, ri)
// persist in-memory state to disk after every 10secs.
logger.LogIf(ctx, ri.updateAfter(ctx, api, 10*time.Second, job))
if success {
break
}
}
}()
}
wk.Wait()
ri.Complete = ri.ObjectsFailed == 0
ri.Failed = ri.ObjectsFailed > 0
globalBatchJobsMetrics.save(job.ID, ri)
// persist in-memory state to disk.
logger.LogIf(ctx, ri.updateAfter(ctx, api, 0, job))
buf, _ := json.Marshal(ri)
if err := r.Notify(ctx, bytes.NewReader(buf)); err != nil {
logger.LogIf(ctx, fmt.Errorf("unable to notify %v", err))
}
cancel()
if ri.Failed {
ri.ObjectsFailed = 0
ri.Bucket = ""
ri.Object = ""
ri.Objects = 0
time.Sleep(delay + time.Duration(rnd.Float64()*float64(delay)))
}
return nil
}
//msgp:ignore batchKeyRotationJobError
type batchKeyRotationJobError struct {
Code string
Description string
HTTPStatusCode int
}
func (e batchKeyRotationJobError) Error() string {
return e.Description
}
// Validate validates the job definition input
func (r *BatchJobKeyRotateV1) Validate(ctx context.Context, job BatchJobRequest, o ObjectLayer) error {
if r == nil {
return nil
}
if r.APIVersion != batchKeyRotateAPIVersion {
return errInvalidArgument
}
if r.Bucket == "" {
return errInvalidArgument
}
if _, err := o.GetBucketInfo(ctx, r.Bucket, BucketOptions{}); err != nil {
if isErrBucketNotFound(err) {
return batchKeyRotationJobError{
Code: "NoSuchSourceBucket",
Description: "The specified source bucket does not exist",
HTTPStatusCode: http.StatusNotFound,
}
}
return err
}
if GlobalKMS == nil {
return errKMSNotConfigured
}
if err := r.Encryption.Validate(); err != nil {
return err
}
for _, tag := range r.Flags.Filter.Tags {
if err := tag.Validate(); err != nil {
return err
}
}
for _, meta := range r.Flags.Filter.Metadata {
if err := meta.Validate(); err != nil {
return err
}
}
if err := r.Flags.Retry.Validate(); err != nil {
return err
}
return nil
}

1650
cmd/batch-rotate_gen.go Normal file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,801 @@
package cmd
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"bytes"
"testing"
"github.com/tinylib/msgp/msgp"
)
func TestMarshalUnmarshalBatchJobKeyRotateEncryption(t *testing.T) {
v := BatchJobKeyRotateEncryption{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgBatchJobKeyRotateEncryption(b *testing.B) {
v := BatchJobKeyRotateEncryption{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBatchJobKeyRotateEncryption(b *testing.B) {
v := BatchJobKeyRotateEncryption{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalBatchJobKeyRotateEncryption(b *testing.B) {
v := BatchJobKeyRotateEncryption{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeBatchJobKeyRotateEncryption(t *testing.T) {
v := BatchJobKeyRotateEncryption{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBatchJobKeyRotateEncryption Msgsize() is inaccurate")
}
vn := BatchJobKeyRotateEncryption{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeBatchJobKeyRotateEncryption(b *testing.B) {
v := BatchJobKeyRotateEncryption{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeBatchJobKeyRotateEncryption(b *testing.B) {
v := BatchJobKeyRotateEncryption{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalBatchJobKeyRotateFlags(t *testing.T) {
v := BatchJobKeyRotateFlags{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgBatchJobKeyRotateFlags(b *testing.B) {
v := BatchJobKeyRotateFlags{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBatchJobKeyRotateFlags(b *testing.B) {
v := BatchJobKeyRotateFlags{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalBatchJobKeyRotateFlags(b *testing.B) {
v := BatchJobKeyRotateFlags{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeBatchJobKeyRotateFlags(t *testing.T) {
v := BatchJobKeyRotateFlags{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBatchJobKeyRotateFlags Msgsize() is inaccurate")
}
vn := BatchJobKeyRotateFlags{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeBatchJobKeyRotateFlags(b *testing.B) {
v := BatchJobKeyRotateFlags{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeBatchJobKeyRotateFlags(b *testing.B) {
v := BatchJobKeyRotateFlags{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalBatchJobKeyRotateV1(t *testing.T) {
v := BatchJobKeyRotateV1{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgBatchJobKeyRotateV1(b *testing.B) {
v := BatchJobKeyRotateV1{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBatchJobKeyRotateV1(b *testing.B) {
v := BatchJobKeyRotateV1{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalBatchJobKeyRotateV1(b *testing.B) {
v := BatchJobKeyRotateV1{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeBatchJobKeyRotateV1(t *testing.T) {
v := BatchJobKeyRotateV1{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBatchJobKeyRotateV1 Msgsize() is inaccurate")
}
vn := BatchJobKeyRotateV1{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeBatchJobKeyRotateV1(b *testing.B) {
v := BatchJobKeyRotateV1{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeBatchJobKeyRotateV1(b *testing.B) {
v := BatchJobKeyRotateV1{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalBatchKeyRotateFilter(t *testing.T) {
v := BatchKeyRotateFilter{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgBatchKeyRotateFilter(b *testing.B) {
v := BatchKeyRotateFilter{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBatchKeyRotateFilter(b *testing.B) {
v := BatchKeyRotateFilter{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalBatchKeyRotateFilter(b *testing.B) {
v := BatchKeyRotateFilter{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeBatchKeyRotateFilter(t *testing.T) {
v := BatchKeyRotateFilter{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBatchKeyRotateFilter Msgsize() is inaccurate")
}
vn := BatchKeyRotateFilter{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeBatchKeyRotateFilter(b *testing.B) {
v := BatchKeyRotateFilter{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeBatchKeyRotateFilter(b *testing.B) {
v := BatchKeyRotateFilter{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalBatchKeyRotateKV(t *testing.T) {
v := BatchKeyRotateKV{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgBatchKeyRotateKV(b *testing.B) {
v := BatchKeyRotateKV{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBatchKeyRotateKV(b *testing.B) {
v := BatchKeyRotateKV{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalBatchKeyRotateKV(b *testing.B) {
v := BatchKeyRotateKV{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeBatchKeyRotateKV(t *testing.T) {
v := BatchKeyRotateKV{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBatchKeyRotateKV Msgsize() is inaccurate")
}
vn := BatchKeyRotateKV{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeBatchKeyRotateKV(b *testing.B) {
v := BatchKeyRotateKV{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeBatchKeyRotateKV(b *testing.B) {
v := BatchKeyRotateKV{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalBatchKeyRotateNotification(t *testing.T) {
v := BatchKeyRotateNotification{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgBatchKeyRotateNotification(b *testing.B) {
v := BatchKeyRotateNotification{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBatchKeyRotateNotification(b *testing.B) {
v := BatchKeyRotateNotification{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalBatchKeyRotateNotification(b *testing.B) {
v := BatchKeyRotateNotification{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeBatchKeyRotateNotification(t *testing.T) {
v := BatchKeyRotateNotification{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBatchKeyRotateNotification Msgsize() is inaccurate")
}
vn := BatchKeyRotateNotification{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeBatchKeyRotateNotification(b *testing.B) {
v := BatchKeyRotateNotification{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeBatchKeyRotateNotification(b *testing.B) {
v := BatchKeyRotateNotification{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalBatchKeyRotateRetry(t *testing.T) {
v := BatchKeyRotateRetry{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgBatchKeyRotateRetry(b *testing.B) {
v := BatchKeyRotateRetry{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBatchKeyRotateRetry(b *testing.B) {
v := BatchKeyRotateRetry{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalBatchKeyRotateRetry(b *testing.B) {
v := BatchKeyRotateRetry{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeBatchKeyRotateRetry(t *testing.T) {
v := BatchKeyRotateRetry{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBatchKeyRotateRetry Msgsize() is inaccurate")
}
vn := BatchKeyRotateRetry{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeBatchKeyRotateRetry(b *testing.B) {
v := BatchKeyRotateRetry{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeBatchKeyRotateRetry(b *testing.B) {
v := BatchKeyRotateRetry{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}

View File

@ -0,0 +1,24 @@
// Code generated by "stringer -type=batchJobMetric -trimprefix=batchJobMetric batch-handlers.go"; DO NOT EDIT.
package cmd
import "strconv"
func _() {
// An "invalid array index" compiler error signifies that the constant values have changed.
// Re-run the stringer command to generate them again.
var x [1]struct{}
_ = x[batchReplicationMetricObject-0]
_ = x[batchKeyRotationMetricObject-1]
}
const _batchJobMetric_name = "batchReplicationMetricObjectbatchKeyRotationMetricObject"
var _batchJobMetric_index = [...]uint8{0, 28, 56}
func (i batchJobMetric) String() string {
if i >= batchJobMetric(len(_batchJobMetric_index)-1) {
return "batchJobMetric(" + strconv.FormatInt(int64(i), 10) + ")"
}
return _batchJobMetric_name[_batchJobMetric_index[i]:_batchJobMetric_index[i+1]]
}

View File

@ -1,23 +0,0 @@
// Code generated by "stringer -type=batchReplicationMetric -trimprefix=batchReplicationMetric batch-handlers.go"; DO NOT EDIT.
package cmd
import "strconv"
func _() {
// An "invalid array index" compiler error signifies that the constant values have changed.
// Re-run the stringer command to generate them again.
var x [1]struct{}
_ = x[batchReplicationMetricObject-0]
}
const _batchReplicationMetric_name = "Object"
var _batchReplicationMetric_index = [...]uint8{0, 6}
func (i batchReplicationMetric) String() string {
if i >= batchReplicationMetric(len(_batchReplicationMetric_index)-1) {
return "batchReplicationMetric(" + strconv.FormatInt(int64(i), 10) + ")"
}
return _batchReplicationMetric_name[_batchReplicationMetric_index[i]:_batchReplicationMetric_index[i+1]]
}

493
go.sum

File diff suppressed because it is too large Load Diff