mirror of
				https://github.com/minio/minio.git
				synced 2025-10-29 15:55:00 -04:00 
			
		
		
		
	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 | ||||
| } | ||||
| 
 | ||||
|  | ||||
							
								
								
									
										564
									
								
								cmd/batch-rotate.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										564
									
								
								cmd/batch-rotate.go
									
									
									
									
									
										Normal 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
									
								
							
							
						
						
									
										1650
									
								
								cmd/batch-rotate_gen.go
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										801
									
								
								cmd/batch-rotate_gen_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										801
									
								
								cmd/batch-rotate_gen_test.go
									
									
									
									
									
										Normal 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) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
							
								
								
									
										24
									
								
								cmd/batchjobmetric_string.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										24
									
								
								cmd/batchjobmetric_string.go
									
									
									
									
									
										Normal 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]] | ||||
| } | ||||
| @ -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…
	
	
			
			x
			
			
		
	
		Reference in New Issue
	
	Block a user