mirror of https://github.com/minio/minio.git
Add support for batch key rotation (#16844)
This commit is contained in:
parent
51f7f9aaa3
commit
3158f2d12e
|
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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]]
|
||||
}
|
|
@ -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]]
|
||||
}
|
Loading…
Reference in New Issue