diff --git a/clientlibrary/checkpoint/dynamodb-checkpointer.go b/clientlibrary/checkpoint/dynamodb-checkpointer.go index 0bf8086..4465f5a 100644 --- a/clientlibrary/checkpoint/dynamodb-checkpointer.go +++ b/clientlibrary/checkpoint/dynamodb-checkpointer.go @@ -30,26 +30,24 @@ package checkpoint import ( + "context" "errors" "fmt" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/client" - "github.com/aws/aws-sdk-go/aws/session" - "github.com/aws/aws-sdk-go/service/dynamodb" - "github.com/aws/aws-sdk-go/service/dynamodb/dynamodbiface" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/aws/retry" + awsConfig "github.com/aws/aws-sdk-go-v2/config" + "github.com/aws/aws-sdk-go-v2/credentials" + "github.com/aws/aws-sdk-go-v2/service/dynamodb" + "github.com/aws/aws-sdk-go-v2/service/dynamodb/types" "github.com/vmware/vmware-go-kcl/clientlibrary/config" par "github.com/vmware/vmware-go-kcl/clientlibrary/partition" - "github.com/vmware/vmware-go-kcl/clientlibrary/utils" "github.com/vmware/vmware-go-kcl/logger" ) const ( - // ErrInvalidDynamoDBSchema is returned when there are one or more fields missing from the table - ErrInvalidDynamoDBSchema = "The DynamoDB schema is invalid and may need to be re-created" - // NumMaxRetries is the max times of doing retry NumMaxRetries = 10 ) @@ -62,8 +60,8 @@ type DynamoCheckpoint struct { leaseTableWriteCapacity int64 LeaseDuration int - svc dynamodbiface.DynamoDBAPI - kclConfig *config.KinesisClientLibConfiguration + svc *dynamodb.Client + kclConfig *config.KinesisClientLibConfiguration Retries int lastLeaseSync time.Time } @@ -83,7 +81,7 @@ func NewDynamoCheckpoint(kclConfig *config.KinesisClientLibConfiguration) *Dynam } // WithDynamoDB is used to provide DynamoDB service -func (checkpointer *DynamoCheckpoint) WithDynamoDB(svc dynamodbiface.DynamoDBAPI) *DynamoCheckpoint { +func (checkpointer *DynamoCheckpoint) WithDynamoDB(svc *dynamodb.Client) *DynamoCheckpoint { checkpointer.svc = svc return checkpointer } @@ -92,31 +90,40 @@ func (checkpointer *DynamoCheckpoint) WithDynamoDB(svc dynamodbiface.DynamoDBAPI func (checkpointer *DynamoCheckpoint) Init() error { checkpointer.log.Infof("Creating DynamoDB session") - s, err := session.NewSession(&aws.Config{ - Region: aws.String(checkpointer.kclConfig.RegionName), - Endpoint: aws.String(checkpointer.kclConfig.DynamoDBEndpoint), - Credentials: checkpointer.kclConfig.DynamoDBCredentials, - Retryer: client.DefaultRetryer{ - NumMaxRetries: checkpointer.Retries, - MinRetryDelay: client.DefaultRetryerMinRetryDelay, - MinThrottleDelay: client.DefaultRetryerMinThrottleDelay, - MaxRetryDelay: client.DefaultRetryerMaxRetryDelay, - MaxThrottleDelay: client.DefaultRetryerMaxRetryDelay, - }, - }) - - if err != nil { - // no need to move forward - checkpointer.log.Fatalf("Failed in getting DynamoDB session for creating Worker: %+v", err) - } - if checkpointer.svc == nil { - checkpointer.svc = dynamodb.New(s) + resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) { + return aws.Endpoint{ + PartitionID: "aws", + URL: checkpointer.kclConfig.DynamoDBEndpoint, + SigningRegion: checkpointer.kclConfig.RegionName, + }, nil + }) + + cfg, err := awsConfig.LoadDefaultConfig( + context.TODO(), + awsConfig.WithRegion(checkpointer.kclConfig.RegionName), + awsConfig.WithCredentialsProvider( + credentials.NewStaticCredentialsProvider( + checkpointer.kclConfig.DynamoDBCredentials.Value.AccessKeyID, + checkpointer.kclConfig.DynamoDBCredentials.Value.SecretAccessKey, + checkpointer.kclConfig.DynamoDBCredentials.Value.SessionToken)), + awsConfig.WithEndpointResolver(resolver), + awsConfig.WithRetryer(func() aws.Retryer { + return retry.AddWithMaxBackoffDelay(retry.NewStandard(), retry.DefaultMaxBackoff) + }), + ) + + if err != nil { + checkpointer.log.Fatalf("unable to load SDK config, %v", err) + } + + checkpointer.svc = dynamodb.NewFromConfig(cfg) } if !checkpointer.doesTableExist() { return checkpointer.createTable() } + return nil } @@ -133,8 +140,12 @@ func (checkpointer *DynamoCheckpoint) GetLease(shard *par.ShardStatus, newAssign var claimRequest string if checkpointer.kclConfig.EnableLeaseStealing { - if currentCheckpointClaimRequest, ok := currentCheckpoint[ClaimRequestKey]; ok && currentCheckpointClaimRequest.S != nil { - claimRequest = *currentCheckpointClaimRequest.S + if currentCheckpointClaimRequest, ok := currentCheckpoint[ClaimRequestKey]; ok { + fmt.Printf("aaaaaa %v", currentCheckpointClaimRequest) + } + if currentCheckpointClaimRequest, ok := currentCheckpoint[ClaimRequestKey]; ok && + currentCheckpointClaimRequest.(*types.AttributeValueMemberS).Value != "" { + claimRequest = currentCheckpointClaimRequest.(*types.AttributeValueMemberS).Value if newAssignTo != claimRequest && !isClaimRequestExpired { checkpointer.log.Debugf("another worker: %s has a claim on this shard. Not going to renew the lease", claimRequest) return errors.New(ErrShardClaimed) @@ -146,13 +157,13 @@ func (checkpointer *DynamoCheckpoint) GetLease(shard *par.ShardStatus, newAssign leaseVar, leaseTimeoutOk := currentCheckpoint[LeaseTimeoutKey] var conditionalExpression string - var expressionAttributeValues map[string]*dynamodb.AttributeValue + var expressionAttributeValues map[string]types.AttributeValue if !leaseTimeoutOk || !assignedToOk { conditionalExpression = "attribute_not_exists(AssignedTo)" } else { - assignedTo := *assignedVar.S - leaseTimeout := *leaseVar.S + assignedTo := assignedVar.(*types.AttributeValueMemberS).Value + leaseTimeout := leaseVar.(*types.AttributeValueMemberS).Value currentLeaseTimeout, err := time.Parse(time.RFC3339, leaseTimeout) if err != nil { @@ -171,57 +182,60 @@ func (checkpointer *DynamoCheckpoint) GetLease(shard *par.ShardStatus, newAssign checkpointer.log.Debugf("Attempting to get a lock for shard: %s, leaseTimeout: %s, assignedTo: %s, newAssignedTo: %s", shard.ID, currentLeaseTimeout, assignedTo, newAssignTo) conditionalExpression = "ShardID = :id AND AssignedTo = :assigned_to AND LeaseTimeout = :lease_timeout" - expressionAttributeValues = map[string]*dynamodb.AttributeValue{ - ":id": { - S: aws.String(shard.ID), + expressionAttributeValues = map[string]types.AttributeValue{ + ":id": &types.AttributeValueMemberS{ + Value: shard.ID, }, - ":assigned_to": { - S: aws.String(assignedTo), + ":assigned_to": &types.AttributeValueMemberS{ + Value: assignedTo, }, - ":lease_timeout": { - S: aws.String(leaseTimeout), + ":lease_timeout": &types.AttributeValueMemberS{ + Value: leaseTimeout, }, } } - marshalledCheckpoint := map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: aws.String(shard.ID), + marshalledCheckpoint := map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: shard.ID, }, - LeaseOwnerKey: { - S: aws.String(newAssignTo), + LeaseOwnerKey: &types.AttributeValueMemberS{ + Value: newAssignTo, }, - LeaseTimeoutKey: { - S: aws.String(newLeaseTimeoutString), + LeaseTimeoutKey: &types.AttributeValueMemberS{ + Value: newLeaseTimeoutString, }, } if len(shard.ParentShardId) > 0 { - marshalledCheckpoint[ParentShardIdKey] = &dynamodb.AttributeValue{S: aws.String(shard.ParentShardId)} + marshalledCheckpoint[ParentShardIdKey] = &types.AttributeValueMemberS{ + Value: shard.ParentShardId, + } } if checkpoint := shard.GetCheckpoint(); checkpoint != "" { - marshalledCheckpoint[SequenceNumberKey] = &dynamodb.AttributeValue{ - S: aws.String(checkpoint), + marshalledCheckpoint[SequenceNumberKey] = &types.AttributeValueMemberS{ + Value: checkpoint, } } if checkpointer.kclConfig.EnableLeaseStealing { if claimRequest != "" && claimRequest == newAssignTo && !isClaimRequestExpired { if expressionAttributeValues == nil { - expressionAttributeValues = make(map[string]*dynamodb.AttributeValue) + expressionAttributeValues = make(map[string]types.AttributeValue) } conditionalExpression = conditionalExpression + " AND ClaimRequest = :claim_request" - expressionAttributeValues[":claim_request"] = &dynamodb.AttributeValue{ - S: &claimRequest, + expressionAttributeValues[":claim_request"] = &types.AttributeValueMemberS{ + Value: claimRequest, } } } err = checkpointer.conditionalUpdate(conditionalExpression, expressionAttributeValues, marshalledCheckpoint) if err != nil { - if utils.AWSErrCode(err) == dynamodb.ErrCodeConditionalCheckFailedException { - return ErrLeaseNotAcquired{dynamodb.ErrCodeConditionalCheckFailedException} + var conditionalCheckErr *types.ConditionalCheckFailedException + if errors.As(err, &conditionalCheckErr) { + return ErrLeaseNotAcquired{conditionalCheckErr.ErrorMessage()} } return err } @@ -237,23 +251,23 @@ func (checkpointer *DynamoCheckpoint) GetLease(shard *par.ShardStatus, newAssign // CheckpointSequence writes a checkpoint at the designated sequence ID func (checkpointer *DynamoCheckpoint) CheckpointSequence(shard *par.ShardStatus) error { leaseTimeout := shard.GetLeaseTimeout().UTC().Format(time.RFC3339) - marshalledCheckpoint := map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: aws.String(shard.ID), + marshalledCheckpoint := map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: shard.ID, }, - SequenceNumberKey: { - S: aws.String(shard.GetCheckpoint()), + SequenceNumberKey: &types.AttributeValueMemberS{ + Value: shard.GetCheckpoint(), }, - LeaseOwnerKey: { - S: aws.String(shard.GetLeaseOwner()), + LeaseOwnerKey: &types.AttributeValueMemberS{ + Value: shard.GetLeaseOwner(), }, - LeaseTimeoutKey: { - S: aws.String(leaseTimeout), + LeaseTimeoutKey: &types.AttributeValueMemberS{ + Value: leaseTimeout, }, } if len(shard.ParentShardId) > 0 { - marshalledCheckpoint[ParentShardIdKey] = &dynamodb.AttributeValue{S: &shard.ParentShardId} + marshalledCheckpoint[ParentShardIdKey] = &types.AttributeValueMemberS{Value: shard.ParentShardId} } return checkpointer.saveItem(marshalledCheckpoint) @@ -270,16 +284,17 @@ func (checkpointer *DynamoCheckpoint) FetchCheckpoint(shard *par.ShardStatus) er if !ok { return ErrSequenceIDNotFound } - checkpointer.log.Debugf("Retrieved Shard Iterator %s", *sequenceID.S) - shard.SetCheckpoint(aws.StringValue(sequenceID.S)) + + checkpointer.log.Debugf("Retrieved Shard Iterator %s", sequenceID.(*types.AttributeValueMemberS).Value) + shard.SetCheckpoint(sequenceID.(*types.AttributeValueMemberS).Value) if assignedTo, ok := checkpoint[LeaseOwnerKey]; ok { - shard.SetLeaseOwner(aws.StringValue(assignedTo.S)) + shard.SetLeaseOwner(assignedTo.(*types.AttributeValueMemberS).Value) } // Use up-to-date leaseTimeout to avoid ConditionalCheckFailedException when claiming - if leaseTimeout, ok := checkpoint[LeaseTimeoutKey]; ok && leaseTimeout.S != nil { - currentLeaseTimeout, err := time.Parse(time.RFC3339, aws.StringValue(leaseTimeout.S)) + if leaseTimeout, ok := checkpoint[LeaseTimeoutKey]; ok && leaseTimeout.(*types.AttributeValueMemberS).Value != "" { + currentLeaseTimeout, err := time.Parse(time.RFC3339, leaseTimeout.(*types.AttributeValueMemberS).Value) if err != nil { return err } @@ -306,21 +321,21 @@ func (checkpointer *DynamoCheckpoint) RemoveLeaseInfo(shardID string) error { func (checkpointer *DynamoCheckpoint) RemoveLeaseOwner(shardID string) error { input := &dynamodb.UpdateItemInput{ TableName: aws.String(checkpointer.TableName), - Key: map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: aws.String(shardID), + Key: map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: shardID, }, }, UpdateExpression: aws.String("remove " + LeaseOwnerKey), - ExpressionAttributeValues: map[string]*dynamodb.AttributeValue{ - ":assigned_to": { - S: aws.String(checkpointer.kclConfig.WorkerID), + ExpressionAttributeValues: map[string]types.AttributeValue{ + ":assigned_to": &types.AttributeValueMemberS{ + Value: checkpointer.kclConfig.WorkerID, }, }, ConditionExpression: aws.String("AssignedTo = :assigned_to"), } - _, err := checkpointer.svc.UpdateItem(input) + _, err := checkpointer.svc.UpdateItem(context.TODO(), input) return err } @@ -343,6 +358,7 @@ func (checkpointer *DynamoCheckpoint) ListActiveWorkers(shardStatus map[string]* checkpointer.log.Debugf("Shard Not Assigned Error. ShardID: %s, WorkerID: %s", shard.ID, checkpointer.kclConfig.WorkerID) return nil, ErrShardNotAssigned } + if w, ok := workers[leaseOwner]; ok { workers[leaseOwner] = append(w, shard) } else { @@ -361,54 +377,54 @@ func (checkpointer *DynamoCheckpoint) ClaimShard(shard *par.ShardStatus, claimID leaseTimeoutString := shard.GetLeaseTimeout().Format(time.RFC3339) conditionalExpression := `ShardID = :id AND LeaseTimeout = :lease_timeout AND attribute_not_exists(ClaimRequest)` - expressionAttributeValues := map[string]*dynamodb.AttributeValue{ - ":id": { - S: aws.String(shard.ID), + expressionAttributeValues := map[string]types.AttributeValue{ + ":id": &types.AttributeValueMemberS{ + Value: shard.ID, }, - ":lease_timeout": { - S: aws.String(leaseTimeoutString), + ":lease_timeout": &types.AttributeValueMemberS{ + Value: leaseTimeoutString, }, } - marshalledCheckpoint := map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: &shard.ID, + marshalledCheckpoint := map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: shard.ID, }, - LeaseTimeoutKey: { - S: &leaseTimeoutString, + LeaseTimeoutKey: &types.AttributeValueMemberS{ + Value: leaseTimeoutString, }, - SequenceNumberKey: { - S: &shard.Checkpoint, + SequenceNumberKey: &types.AttributeValueMemberS{ + Value: shard.Checkpoint, }, - ClaimRequestKey: { - S: &claimID, + ClaimRequestKey: &types.AttributeValueMemberS{ + Value: claimID, }, } if leaseOwner := shard.GetLeaseOwner(); leaseOwner == "" { conditionalExpression += " AND attribute_not_exists(AssignedTo)" } else { - marshalledCheckpoint[LeaseOwnerKey] = &dynamodb.AttributeValue{S: &leaseOwner} + marshalledCheckpoint[LeaseOwnerKey] = &types.AttributeValueMemberS{Value: leaseOwner} conditionalExpression += "AND AssignedTo = :assigned_to" - expressionAttributeValues[":assigned_to"] = &dynamodb.AttributeValue{S: &leaseOwner} + expressionAttributeValues[":assigned_to"] = &types.AttributeValueMemberS{Value: leaseOwner} } if checkpoint := shard.GetCheckpoint(); checkpoint == "" { conditionalExpression += " AND attribute_not_exists(Checkpoint)" } else if checkpoint == ShardEnd { conditionalExpression += " AND Checkpoint <> :checkpoint" - expressionAttributeValues[":checkpoint"] = &dynamodb.AttributeValue{S: aws.String(ShardEnd)} + expressionAttributeValues[":checkpoint"] = &types.AttributeValueMemberS{Value: ShardEnd} } else { conditionalExpression += " AND Checkpoint = :checkpoint" - expressionAttributeValues[":checkpoint"] = &dynamodb.AttributeValue{S: &checkpoint} + expressionAttributeValues[":checkpoint"] = &types.AttributeValueMemberS{Value: checkpoint} } if shard.ParentShardId == "" { conditionalExpression += " AND attribute_not_exists(ParentShardId)" } else { - marshalledCheckpoint[ParentShardIdKey] = &dynamodb.AttributeValue{S: aws.String(shard.ParentShardId)} + marshalledCheckpoint[ParentShardIdKey] = &types.AttributeValueMemberS{Value: shard.ParentShardId} conditionalExpression += " AND ParentShardId = :parent_shard" - expressionAttributeValues[":parent_shard"] = &dynamodb.AttributeValue{S: &shard.ParentShardId} + expressionAttributeValues[":parent_shard"] = &types.AttributeValueMemberS{Value: shard.ParentShardId} } return checkpointer.conditionalUpdate(conditionalExpression, expressionAttributeValues, marshalledCheckpoint) @@ -424,27 +440,25 @@ func (checkpointer *DynamoCheckpoint) syncLeases(shardStatus map[string]*par.Sha checkpointer.lastLeaseSync = time.Now() input := &dynamodb.ScanInput{ ProjectionExpression: aws.String(fmt.Sprintf("%s,%s,%s", LeaseKeyKey, LeaseOwnerKey, SequenceNumberKey)), - Select: aws.String("SPECIFIC_ATTRIBUTES"), + Select: "SPECIFIC_ATTRIBUTES", TableName: aws.String(checkpointer.kclConfig.TableName), } - err := checkpointer.svc.ScanPages(input, - func(pages *dynamodb.ScanOutput, lastPage bool) bool { - results := pages.Items - for _, result := range results { - shardId, foundShardId := result[LeaseKeyKey] - assignedTo, foundAssignedTo := result[LeaseOwnerKey] - checkpoint, foundCheckpoint := result[SequenceNumberKey] - if !foundShardId || !foundAssignedTo || !foundCheckpoint { - continue - } - if shard, ok := shardStatus[aws.StringValue(shardId.S)]; ok { - shard.SetLeaseOwner(aws.StringValue(assignedTo.S)) - shard.SetCheckpoint(aws.StringValue(checkpoint.S)) - } - } - return !lastPage - }) + scanOutput, err := checkpointer.svc.Scan(context.TODO(), input) + results := scanOutput.Items + for _, result := range results { + shardId, foundShardId := result[LeaseKeyKey] + assignedTo, foundAssignedTo := result[LeaseOwnerKey] + checkpoint, foundCheckpoint := result[SequenceNumberKey] + if !foundShardId || !foundAssignedTo || !foundCheckpoint { + continue + } + + if shard, ok := shardStatus[shardId.(*types.AttributeValueMemberS).Value]; ok { + shard.SetLeaseOwner(assignedTo.(*types.AttributeValueMemberS).Value) + shard.SetCheckpoint(checkpoint.(*types.AttributeValueMemberS).Value) + } + } if err != nil { log.Debugf("Error performing SyncLeases. Error: %+v ", err) @@ -456,25 +470,26 @@ func (checkpointer *DynamoCheckpoint) syncLeases(shardStatus map[string]*par.Sha func (checkpointer *DynamoCheckpoint) createTable() error { input := &dynamodb.CreateTableInput{ - AttributeDefinitions: []*dynamodb.AttributeDefinition{ + AttributeDefinitions: []types.AttributeDefinition{ { AttributeName: aws.String(LeaseKeyKey), - AttributeType: aws.String("S"), + AttributeType: types.ScalarAttributeTypeS, }, }, - KeySchema: []*dynamodb.KeySchemaElement{ + KeySchema: []types.KeySchemaElement{ { AttributeName: aws.String(LeaseKeyKey), - KeyType: aws.String("HASH"), + KeyType: types.KeyTypeHash, }, }, - ProvisionedThroughput: &dynamodb.ProvisionedThroughput{ + ProvisionedThroughput: &types.ProvisionedThroughput{ ReadCapacityUnits: aws.Int64(checkpointer.leaseTableReadCapacity), WriteCapacityUnits: aws.Int64(checkpointer.leaseTableWriteCapacity), }, TableName: aws.String(checkpointer.TableName), } - _, err := checkpointer.svc.CreateTable(input) + _, err := checkpointer.svc.CreateTable(context.Background(), input) + return err } @@ -482,18 +497,19 @@ func (checkpointer *DynamoCheckpoint) doesTableExist() bool { input := &dynamodb.DescribeTableInput{ TableName: aws.String(checkpointer.TableName), } - _, err := checkpointer.svc.DescribeTable(input) + _, err := checkpointer.svc.DescribeTable(context.Background(), input) + return err == nil } -func (checkpointer *DynamoCheckpoint) saveItem(item map[string]*dynamodb.AttributeValue) error { +func (checkpointer *DynamoCheckpoint) saveItem(item map[string]types.AttributeValue) error { return checkpointer.putItem(&dynamodb.PutItemInput{ TableName: aws.String(checkpointer.TableName), Item: item, }) } -func (checkpointer *DynamoCheckpoint) conditionalUpdate(conditionExpression string, expressionAttributeValues map[string]*dynamodb.AttributeValue, item map[string]*dynamodb.AttributeValue) error { +func (checkpointer *DynamoCheckpoint) conditionalUpdate(conditionExpression string, expressionAttributeValues map[string]types.AttributeValue, item map[string]types.AttributeValue) error { return checkpointer.putItem(&dynamodb.PutItemInput{ ConditionExpression: aws.String(conditionExpression), TableName: aws.String(checkpointer.TableName), @@ -503,30 +519,38 @@ func (checkpointer *DynamoCheckpoint) conditionalUpdate(conditionExpression stri } func (checkpointer *DynamoCheckpoint) putItem(input *dynamodb.PutItemInput) error { - _, err := checkpointer.svc.PutItem(input) + _, err := checkpointer.svc.PutItem(context.Background(), input) return err } -func (checkpointer *DynamoCheckpoint) getItem(shardID string) (map[string]*dynamodb.AttributeValue, error) { - item, err := checkpointer.svc.GetItem(&dynamodb.GetItemInput{ - TableName: aws.String(checkpointer.TableName), - Key: map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: aws.String(shardID), +func (checkpointer *DynamoCheckpoint) getItem(shardID string) (map[string]types.AttributeValue, error) { + item, err := checkpointer.svc.GetItem(context.Background(), &dynamodb.GetItemInput{ + TableName: aws.String(checkpointer.TableName), + ConsistentRead: aws.Bool(true), + Key: map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: shardID, }, }, }) + + // fix problem when starts the environment from scratch (dynamo table is empty) + if item == nil { + return nil, err + } + return item.Item, err } func (checkpointer *DynamoCheckpoint) removeItem(shardID string) error { - _, err := checkpointer.svc.DeleteItem(&dynamodb.DeleteItemInput{ + _, err := checkpointer.svc.DeleteItem(context.Background(), &dynamodb.DeleteItemInput{ TableName: aws.String(checkpointer.TableName), - Key: map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: aws.String(shardID), + Key: map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: shardID, }, }, }) + return err } diff --git a/clientlibrary/checkpoint/dynamodb-checkpointer_test.go b/clientlibrary/checkpoint/dynamodb-checkpointer_test.go index 38da0b3..1f0a8f1 100644 --- a/clientlibrary/checkpoint/dynamodb-checkpointer_test.go +++ b/clientlibrary/checkpoint/dynamodb-checkpointer_test.go @@ -28,15 +28,15 @@ package checkpoint import ( + "context" "errors" "sync" "testing" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/awserr" - "github.com/aws/aws-sdk-go/service/dynamodb" - "github.com/aws/aws-sdk-go/service/dynamodb/dynamodbiface" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/dynamodb" + "github.com/aws/aws-sdk-go-v2/service/dynamodb/types" "github.com/stretchr/testify/assert" cfg "github.com/vmware/vmware-go-kcl/clientlibrary/config" @@ -44,24 +44,24 @@ import ( ) func TestDoesTableExist(t *testing.T) { - svc := &mockDynamoDB{tableExist: true, item: map[string]*dynamodb.AttributeValue{}} + svc := &mockDynamoDB{client: nil, tableExist: true, item: map[string]types.AttributeValue{}} checkpoint := &DynamoCheckpoint{ TableName: "TableName", - svc: svc, + svc: svc.client, } if !checkpoint.doesTableExist() { t.Error("Table exists but returned false") } svc = &mockDynamoDB{tableExist: false} - checkpoint.svc = svc + checkpoint.svc = svc.client if checkpoint.doesTableExist() { t.Error("Table does not exist but returned true") } } -func TestGetLeaseNotAquired(t *testing.T) { - svc := &mockDynamoDB{tableExist: true, item: map[string]*dynamodb.AttributeValue{}} +func TestGetLeaseNotAcquired(t *testing.T) { + svc := &mockDynamoDB{tableExist: true, item: map[string]types.AttributeValue{}} kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). WithInitialPositionInStream(cfg.LATEST). WithMaxRecords(10). @@ -69,8 +69,8 @@ func TestGetLeaseNotAquired(t *testing.T) { WithShardSyncIntervalMillis(5000). WithFailoverTimeMillis(300000) - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() err := checkpoint.GetLease(&par.ShardStatus{ ID: "0001", Checkpoint: "", @@ -92,7 +92,7 @@ func TestGetLeaseNotAquired(t *testing.T) { } func TestGetLeaseAquired(t *testing.T) { - svc := &mockDynamoDB{tableExist: true, item: map[string]*dynamodb.AttributeValue{}} + svc := &mockDynamoDB{tableExist: true, item: map[string]types.AttributeValue{}} kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). WithInitialPositionInStream(cfg.LATEST). WithMaxRecords(10). @@ -100,27 +100,28 @@ func TestGetLeaseAquired(t *testing.T) { WithShardSyncIntervalMillis(5000). WithFailoverTimeMillis(300000) - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() - marshalledCheckpoint := map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: aws.String("0001"), + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() + marshalledCheckpoint := map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: "0001", }, - LeaseOwnerKey: { - S: aws.String("abcd-efgh"), + LeaseOwnerKey: &types.AttributeValueMemberS{ + Value: "abcd-efgh", }, - LeaseTimeoutKey: { - S: aws.String(time.Now().AddDate(0, -1, 0).UTC().Format(time.RFC3339)), + LeaseTimeoutKey: &types.AttributeValueMemberS{ + Value: time.Now().AddDate(0, -1, 0).UTC().Format(time.RFC3339), }, - SequenceNumberKey: { - S: aws.String("deadbeef"), + SequenceNumberKey: &types.AttributeValueMemberS{ + Value: "deadbeef", }, } input := &dynamodb.PutItemInput{ TableName: aws.String("TableName"), Item: marshalledCheckpoint, } - checkpoint.svc.PutItem(input) + _, _ = checkpoint.svc.PutItem(context.TODO(), input) + shard := &par.ShardStatus{ ID: "0001", Checkpoint: "deadbeef", @@ -135,8 +136,8 @@ func TestGetLeaseAquired(t *testing.T) { id, ok := svc.item[SequenceNumberKey] if !ok { t.Error("Expected checkpoint to be set by GetLease") - } else if *id.S != "deadbeef" { - t.Errorf("Expected checkpoint to be deadbeef. Got '%s'", *id.S) + } else if id.(*types.AttributeValueMemberS).Value != "deadbeef" { + t.Errorf("Expected checkpoint to be deadbeef. Got '%s'", id.(*types.AttributeValueMemberS).Value) } // release owner info @@ -147,7 +148,7 @@ func TestGetLeaseAquired(t *testing.T) { ID: shard.ID, Mux: &sync.RWMutex{}, } - checkpoint.FetchCheckpoint(status) + _ = checkpoint.FetchCheckpoint(status) // checkpointer and parent shard id should be the same assert.Equal(t, shard.Checkpoint, status.Checkpoint) @@ -161,9 +162,9 @@ func TestGetLeaseShardClaimed(t *testing.T) { leaseTimeout := time.Now().Add(-100 * time.Second).UTC() svc := &mockDynamoDB{ tableExist: true, - item: map[string]*dynamodb.AttributeValue{ - ClaimRequestKey: {S: aws.String("ijkl-mnop")}, - LeaseTimeoutKey: {S: aws.String(leaseTimeout.Format(time.RFC3339))}, + item: map[string]types.AttributeValue{ + ClaimRequestKey: &types.AttributeValueMemberS{Value: "ijkl-mnop"}, + LeaseTimeoutKey: &types.AttributeValueMemberS{Value: leaseTimeout.Format(time.RFC3339)}, }, } kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). @@ -174,8 +175,8 @@ func TestGetLeaseShardClaimed(t *testing.T) { WithFailoverTimeMillis(300000). WithLeaseStealing(true) - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() err := checkpoint.GetLease(&par.ShardStatus{ ID: "0001", Checkpoint: "", @@ -214,15 +215,15 @@ func TestGetLeaseClaimRequestExpiredOwner(t *testing.T) { svc := &mockDynamoDB{ tableExist: true, - item: map[string]*dynamodb.AttributeValue{ - LeaseOwnerKey: {S: aws.String("abcd-efgh")}, - ClaimRequestKey: {S: aws.String("ijkl-mnop")}, - LeaseTimeoutKey: {S: aws.String(leaseTimeout.Format(time.RFC3339))}, + item: map[string]types.AttributeValue{ + LeaseOwnerKey: &types.AttributeValueMemberS{Value: "abcd-efgh"}, + ClaimRequestKey: &types.AttributeValueMemberS{Value: "ijkl-mnop"}, + LeaseTimeoutKey: &types.AttributeValueMemberS{Value: leaseTimeout.Format(time.RFC3339)}, }, } - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() err := checkpoint.GetLease(&par.ShardStatus{ ID: "0001", Checkpoint: "", @@ -251,15 +252,15 @@ func TestGetLeaseClaimRequestExpiredClaimer(t *testing.T) { svc := &mockDynamoDB{ tableExist: true, - item: map[string]*dynamodb.AttributeValue{ - LeaseOwnerKey: {S: aws.String("abcd-efgh")}, - ClaimRequestKey: {S: aws.String("ijkl-mnop")}, - LeaseTimeoutKey: {S: aws.String(leaseTimeout.Format(time.RFC3339))}, + item: map[string]types.AttributeValue{ + LeaseOwnerKey: &types.AttributeValueMemberS{Value: "abcd-efgh"}, + ClaimRequestKey: &types.AttributeValueMemberS{Value: "ijkl-mnop"}, + LeaseTimeoutKey: &types.AttributeValueMemberS{Value: leaseTimeout.Format(time.RFC3339)}, }, } - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() err := checkpoint.GetLease(&par.ShardStatus{ ID: "0001", Checkpoint: "", @@ -276,11 +277,11 @@ func TestFetchCheckpointWithStealing(t *testing.T) { svc := &mockDynamoDB{ tableExist: true, - item: map[string]*dynamodb.AttributeValue{ - SequenceNumberKey: {S: aws.String("deadbeef")}, - LeaseOwnerKey: {S: aws.String("abcd-efgh")}, - LeaseTimeoutKey: { - S: aws.String(future.Format(time.RFC3339)), + item: map[string]types.AttributeValue{ + SequenceNumberKey: &types.AttributeValueMemberS{Value: "deadbeef"}, + LeaseOwnerKey: &types.AttributeValueMemberS{Value: "abcd-efgh"}, + LeaseTimeoutKey: &types.AttributeValueMemberS{ + Value: future.Format(time.RFC3339), }, }, } @@ -293,8 +294,8 @@ func TestFetchCheckpointWithStealing(t *testing.T) { WithFailoverTimeMillis(300000). WithLeaseStealing(true) - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() status := &par.ShardStatus{ ID: "0001", @@ -303,14 +304,14 @@ func TestFetchCheckpointWithStealing(t *testing.T) { Mux: &sync.RWMutex{}, } - checkpoint.FetchCheckpoint(status) + _ = checkpoint.FetchCheckpoint(status) - leaseTimeout, _ := time.Parse(time.RFC3339, *svc.item[LeaseTimeoutKey].S) + leaseTimeout, _ := time.Parse(time.RFC3339, svc.item[LeaseTimeoutKey].(*types.AttributeValueMemberS).Value) assert.Equal(t, leaseTimeout, status.LeaseTimeout) } func TestGetLeaseConditional(t *testing.T) { - svc := &mockDynamoDB{tableExist: true, item: map[string]*dynamodb.AttributeValue{}} + svc := &mockDynamoDB{tableExist: true, item: map[string]types.AttributeValue{}} kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). WithInitialPositionInStream(cfg.LATEST). WithMaxRecords(10). @@ -319,30 +320,31 @@ func TestGetLeaseConditional(t *testing.T) { WithFailoverTimeMillis(300000). WithLeaseStealing(true) - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() - marshalledCheckpoint := map[string]*dynamodb.AttributeValue{ - LeaseKeyKey: { - S: aws.String("0001"), + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() + marshalledCheckpoint := map[string]types.AttributeValue{ + LeaseKeyKey: &types.AttributeValueMemberS{ + Value: "0001", }, - LeaseOwnerKey: { - S: aws.String("abcd-efgh"), + LeaseOwnerKey: &types.AttributeValueMemberS{ + Value: "abcd-efgh", }, - LeaseTimeoutKey: { - S: aws.String(time.Now().Add(-1 * time.Second).UTC().Format(time.RFC3339)), + LeaseTimeoutKey: &types.AttributeValueMemberS{ + Value: time.Now().Add(-1 * time.Second).UTC().Format(time.RFC3339), }, - SequenceNumberKey: { - S: aws.String("deadbeef"), + SequenceNumberKey: &types.AttributeValueMemberS{ + Value: "deadbeef", }, - ClaimRequestKey: { - S: aws.String("ijkl-mnop"), + ClaimRequestKey: &types.AttributeValueMemberS{ + Value: "ijkl-mnop", }, } input := &dynamodb.PutItemInput{ TableName: aws.String("TableName"), Item: marshalledCheckpoint, } - checkpoint.svc.PutItem(input) + _, _ = checkpoint.svc.PutItem(context.TODO(), input) + shard := &par.ShardStatus{ ID: "0001", Checkpoint: "deadbeef", @@ -358,26 +360,149 @@ func TestGetLeaseConditional(t *testing.T) { if err != nil { t.Errorf("Lease not aquired after timeout %s", err) } - assert.Equal(t, *svc.expressionAttributeValues[":claim_request"].S, "ijkl-mnop") + assert.Equal(t, svc.expressionAttributeValues[":claim_request"].(*types.AttributeValueMemberS).Value, "ijkl-mnop") assert.Contains(t, svc.conditionalExpression, " AND ClaimRequest = :claim_request") } -type mockDynamoDB struct { - dynamodbiface.DynamoDBAPI - tableExist bool - item map[string]*dynamodb.AttributeValue - conditionalExpression string - expressionAttributeValues map[string]*dynamodb.AttributeValue +func TestListActiveWorkers(t *testing.T) { + svc := &mockDynamoDB{tableExist: true, item: map[string]types.AttributeValue{}} + kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). + WithLeaseStealing(true) + + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + err := checkpoint.Init() + if err != nil { + t.Errorf("Checkpoint initialization failed: %+v", err) + } + + shardStatus := map[string]*par.ShardStatus{ + "0000": {ID: "0000", AssignedTo: "worker_1", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0001": {ID: "0001", AssignedTo: "worker_2", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0002": {ID: "0002", AssignedTo: "worker_4", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0003": {ID: "0003", AssignedTo: "worker_0", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0004": {ID: "0004", AssignedTo: "worker_1", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0005": {ID: "0005", AssignedTo: "worker_3", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0006": {ID: "0006", AssignedTo: "worker_3", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0007": {ID: "0007", AssignedTo: "worker_0", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0008": {ID: "0008", AssignedTo: "worker_4", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0009": {ID: "0009", AssignedTo: "worker_2", Checkpoint: "", Mux: &sync.RWMutex{}}, + "0010": {ID: "0010", AssignedTo: "worker_0", Checkpoint: ShardEnd, Mux: &sync.RWMutex{}}, + } + + workers, err := checkpoint.ListActiveWorkers(shardStatus) + if err != nil { + t.Error(err) + } + + for workerID, shards := range workers { + assert.Equal(t, 2, len(shards)) + for _, shard := range shards { + assert.Equal(t, workerID, shard.AssignedTo) + } + } } -func (m *mockDynamoDB) ScanPages(*dynamodb.ScanInput, func(*dynamodb.ScanOutput, bool) bool) error { +func TestListActiveWorkersErrShardNotAssigned(t *testing.T) { + svc := &mockDynamoDB{tableExist: true, item: map[string]types.AttributeValue{}} + kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). + WithLeaseStealing(true) + + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + err := checkpoint.Init() + if err != nil { + t.Errorf("Checkpoint initialization failed: %+v", err) + } + + shardStatus := map[string]*par.ShardStatus{ + "0000": {ID: "0000", Mux: &sync.RWMutex{}}, + } + + _, err = checkpoint.ListActiveWorkers(shardStatus) + if err != ErrShardNotAssigned { + t.Error("Expected ErrShardNotAssigned when shard is missing AssignedTo value") + } +} + +func TestClaimShard(t *testing.T) { + svc := &mockDynamoDB{tableExist: true, item: map[string]types.AttributeValue{}} + kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). + WithInitialPositionInStream(cfg.LATEST). + WithMaxRecords(10). + WithMaxLeasesForWorker(1). + WithShardSyncIntervalMillis(5000). + WithFailoverTimeMillis(300000). + WithLeaseStealing(true) + + checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc.client) + _ = checkpoint.Init() + + marshalledCheckpoint := map[string]types.AttributeValue{ + "ShardID": &types.AttributeValueMemberS{ + Value: "0001", + }, + "AssignedTo": &types.AttributeValueMemberS{ + Value: "abcd-efgh", + }, + "LeaseTimeout": &types.AttributeValueMemberS{ + Value: time.Now().AddDate(0, -1, 0).UTC().Format(time.RFC3339), + }, + "Checkpoint": &types.AttributeValueMemberS{ + Value: "deadbeef", + }, + } + input := &dynamodb.PutItemInput{ + TableName: aws.String("TableName"), + Item: marshalledCheckpoint, + } + _, _ = checkpoint.svc.PutItem(context.TODO(), input) + + shard := &par.ShardStatus{ + ID: "0001", + Checkpoint: "deadbeef", + Mux: &sync.RWMutex{}, + } + + err := checkpoint.ClaimShard(shard, "ijkl-mnop") + if err != nil { + t.Errorf("Shard not claimed %s", err) + } + + claimRequest, ok := svc.item[ClaimRequestKey] + if !ok { + t.Error("Expected claimRequest to be set by ClaimShard") + } else if claimRequest.(*types.AttributeValueMemberS).Value != "ijkl-mnop" { + t.Errorf("Expected checkpoint to be ijkl-mnop. Got '%s'", claimRequest.(*types.AttributeValueMemberS).Value) + } + + status := &par.ShardStatus{ + ID: shard.ID, + Mux: &sync.RWMutex{}, + } + _ = checkpoint.FetchCheckpoint(status) + + // asiggnedTo, checkpointer, and parent shard id should be the same + assert.Equal(t, shard.AssignedTo, status.AssignedTo) + assert.Equal(t, shard.Checkpoint, status.Checkpoint) + assert.Equal(t, shard.ParentShardId, status.ParentShardId) +} + +type mockDynamoDB struct { + client *dynamodb.Client + tableExist bool + item map[string]types.AttributeValue + conditionalExpression string + expressionAttributeValues map[string]types.AttributeValue +} + +func (m *mockDynamoDB) ScanPages(_ *dynamodb.ScanInput, _ func(*dynamodb.ScanOutput, bool) bool) error { return nil } -func (m *mockDynamoDB) DescribeTable(*dynamodb.DescribeTableInput) (*dynamodb.DescribeTableOutput, error) { +func (m *mockDynamoDB) DescribeTable(_ *dynamodb.DescribeTableInput) (*dynamodb.DescribeTableOutput, error) { if !m.tableExist { - return &dynamodb.DescribeTableOutput{}, awserr.New(dynamodb.ErrCodeResourceNotFoundException, "doesNotExist", errors.New("")) + return &dynamodb.DescribeTableOutput{}, &types.ResourceNotFoundException{Message: aws.String("doesNotExist")} } + return &dynamodb.DescribeTableOutput{}, nil } @@ -417,7 +542,7 @@ func (m *mockDynamoDB) PutItem(input *dynamodb.PutItemInput) (*dynamodb.PutItemO return nil, nil } -func (m *mockDynamoDB) GetItem(input *dynamodb.GetItemInput) (*dynamodb.GetItemOutput, error) { +func (m *mockDynamoDB) GetItem(_ *dynamodb.GetItemInput) (*dynamodb.GetItemOutput, error) { return &dynamodb.GetItemOutput{ Item: m.item, }, nil @@ -426,134 +551,13 @@ func (m *mockDynamoDB) GetItem(input *dynamodb.GetItemInput) (*dynamodb.GetItemO func (m *mockDynamoDB) UpdateItem(input *dynamodb.UpdateItemInput) (*dynamodb.UpdateItemOutput, error) { exp := input.UpdateExpression - if aws.StringValue(exp) == "remove "+LeaseOwnerKey { + if aws.ToString(exp) == "remove "+LeaseOwnerKey { delete(m.item, LeaseOwnerKey) } return nil, nil } -func (m *mockDynamoDB) CreateTable(input *dynamodb.CreateTableInput) (*dynamodb.CreateTableOutput, error) { +func (m *mockDynamoDB) CreateTable(_ *dynamodb.CreateTableInput) (*dynamodb.CreateTableOutput, error) { return &dynamodb.CreateTableOutput{}, nil } - -func TestListActiveWorkers(t *testing.T) { - svc := &mockDynamoDB{tableExist: true, item: map[string]*dynamodb.AttributeValue{}} - kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). - WithLeaseStealing(true) - - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - err := checkpoint.Init() - if err != nil { - t.Errorf("Checkpoint initialization failed: %+v", err) - } - - shardStatus := map[string]*par.ShardStatus{ - "0000": {ID: "0000", AssignedTo: "worker_1", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0001": {ID: "0001", AssignedTo: "worker_2", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0002": {ID: "0002", AssignedTo: "worker_4", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0003": {ID: "0003", AssignedTo: "worker_0", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0004": {ID: "0004", AssignedTo: "worker_1", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0005": {ID: "0005", AssignedTo: "worker_3", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0006": {ID: "0006", AssignedTo: "worker_3", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0007": {ID: "0007", AssignedTo: "worker_0", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0008": {ID: "0008", AssignedTo: "worker_4", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0009": {ID: "0009", AssignedTo: "worker_2", Checkpoint: "", Mux: &sync.RWMutex{}}, - "0010": {ID: "0010", AssignedTo: "worker_0", Checkpoint: ShardEnd, Mux: &sync.RWMutex{}}, - } - - workers, err := checkpoint.ListActiveWorkers(shardStatus) - if err != nil { - t.Error(err) - } - - for workerID, shards := range workers { - assert.Equal(t, 2, len(shards)) - for _, shard := range shards { - assert.Equal(t, workerID, shard.AssignedTo) - } - } -} - -func TestListActiveWorkersErrShardNotAssigned(t *testing.T) { - svc := &mockDynamoDB{tableExist: true, item: map[string]*dynamodb.AttributeValue{}} - kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). - WithLeaseStealing(true) - - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - err := checkpoint.Init() - if err != nil { - t.Errorf("Checkpoint initialization failed: %+v", err) - } - - shardStatus := map[string]*par.ShardStatus{ - "0000": {ID: "0000", Mux: &sync.RWMutex{}}, - } - - _, err = checkpoint.ListActiveWorkers(shardStatus) - if err != ErrShardNotAssigned { - t.Error("Expected ErrShardNotAssigned when shard is missing AssignedTo value") - } -} - -func TestClaimShard(t *testing.T) { - svc := &mockDynamoDB{tableExist: true, item: map[string]*dynamodb.AttributeValue{}} - kclConfig := cfg.NewKinesisClientLibConfig("appName", "test", "us-west-2", "abc"). - WithInitialPositionInStream(cfg.LATEST). - WithMaxRecords(10). - WithMaxLeasesForWorker(1). - WithShardSyncIntervalMillis(5000). - WithFailoverTimeMillis(300000). - WithLeaseStealing(true) - - checkpoint := NewDynamoCheckpoint(kclConfig).WithDynamoDB(svc) - checkpoint.Init() - - marshalledCheckpoint := map[string]*dynamodb.AttributeValue{ - "ShardID": { - S: aws.String("0001"), - }, - "AssignedTo": { - S: aws.String("abcd-efgh"), - }, - "LeaseTimeout": { - S: aws.String(time.Now().AddDate(0, -1, 0).UTC().Format(time.RFC3339)), - }, - "Checkpoint": { - S: aws.String("deadbeef"), - }, - } - input := &dynamodb.PutItemInput{ - TableName: aws.String("TableName"), - Item: marshalledCheckpoint, - } - checkpoint.svc.PutItem(input) - shard := &par.ShardStatus{ - ID: "0001", - Checkpoint: "deadbeef", - Mux: &sync.RWMutex{}, - } - - err := checkpoint.ClaimShard(shard, "ijkl-mnop") - if err != nil { - t.Errorf("Shard not claimed %s", err) - } - - claimRequest, ok := svc.item[ClaimRequestKey] - if !ok { - t.Error("Expected claimRequest to be set by ClaimShard") - } else if *claimRequest.S != "ijkl-mnop" { - t.Errorf("Expected checkpoint to be ijkl-mnop. Got '%s'", *claimRequest.S) - } - - status := &par.ShardStatus{ - ID: shard.ID, - Mux: &sync.RWMutex{}, - } - checkpoint.FetchCheckpoint(status) - - // asiggnedTo, checkpointer, and parent shard id should be the same - assert.Equal(t, shard.AssignedTo, status.AssignedTo) - assert.Equal(t, shard.Checkpoint, status.Checkpoint) - assert.Equal(t, shard.ParentShardId, status.ParentShardId) -} diff --git a/clientlibrary/config/config.go b/clientlibrary/config/config.go index f6e061d..35cb6ad 100644 --- a/clientlibrary/config/config.go +++ b/clientlibrary/config/config.go @@ -41,8 +41,8 @@ import ( "strings" "time" - "github.com/aws/aws-sdk-go/aws" - creds "github.com/aws/aws-sdk-go/aws/credentials" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/credentials" "github.com/vmware/vmware-go-kcl/clientlibrary/metrics" "github.com/vmware/vmware-go-kcl/logger" @@ -173,10 +173,10 @@ type ( KinesisEndpoint string // KinesisCredentials is used to access Kinesis - KinesisCredentials *creds.Credentials + KinesisCredentials *credentials.StaticCredentialsProvider // DynamoDBCredentials is used to access DynamoDB - DynamoDBCredentials *creds.Credentials + DynamoDBCredentials *credentials.StaticCredentialsProvider // TableName is name of the dynamo db table for managing kinesis stream default to ApplicationName TableName string diff --git a/clientlibrary/config/kcl-config.go b/clientlibrary/config/kcl-config.go index 6751f6a..45a6a2a 100644 --- a/clientlibrary/config/kcl-config.go +++ b/clientlibrary/config/kcl-config.go @@ -39,7 +39,7 @@ import ( "log" "time" - "github.com/aws/aws-sdk-go/aws/credentials" + "github.com/aws/aws-sdk-go-v2/credentials" "github.com/vmware/vmware-go-kcl/clientlibrary/metrics" "github.com/vmware/vmware-go-kcl/clientlibrary/utils" @@ -54,13 +54,13 @@ func NewKinesisClientLibConfig(applicationName, streamName, regionName, workerID // NewKinesisClientLibConfigWithCredential creates a default KinesisClientLibConfiguration based on the required fields and unique credentials. func NewKinesisClientLibConfigWithCredential(applicationName, streamName, regionName, workerID string, - creds *credentials.Credentials) *KinesisClientLibConfiguration { + creds *credentials.StaticCredentialsProvider) *KinesisClientLibConfiguration { return NewKinesisClientLibConfigWithCredentials(applicationName, streamName, regionName, workerID, creds, creds) } // NewKinesisClientLibConfigWithCredentials creates a default KinesisClientLibConfiguration based on the required fields and specific credentials for each service. func NewKinesisClientLibConfigWithCredentials(applicationName, streamName, regionName, workerID string, - kiniesisCreds, dynamodbCreds *credentials.Credentials) *KinesisClientLibConfiguration { + kinesisCreds, dynamodbCreds *credentials.StaticCredentialsProvider) *KinesisClientLibConfiguration { checkIsValueNotEmpty("ApplicationName", applicationName) checkIsValueNotEmpty("StreamName", streamName) checkIsValueNotEmpty("RegionName", regionName) @@ -70,31 +70,31 @@ func NewKinesisClientLibConfigWithCredentials(applicationName, streamName, regio } // populate the KCL configuration with default values - return &KinesisClientLibConfiguration{ - ApplicationName: applicationName, - KinesisCredentials: kiniesisCreds, - DynamoDBCredentials: dynamodbCreds, - TableName: applicationName, - EnhancedFanOutConsumerName: applicationName, - StreamName: streamName, - RegionName: regionName, - WorkerID: workerID, - InitialPositionInStream: DefaultInitialPositionInStream, - InitialPositionInStreamExtended: *newInitialPosition(DefaultInitialPositionInStream), - FailoverTimeMillis: DefaultFailoverTimeMillis, - LeaseRefreshPeriodMillis: DefaultLeaseRefreshPeriodMillis, - MaxRecords: DefaultMaxRecords, - IdleTimeBetweenReadsInMillis: DefaultIdletimeBetweenReadsMillis, - CallProcessRecordsEvenForEmptyRecordList: DefaultDontCallProcessRecordsForEmptyRecordList, - ParentShardPollIntervalMillis: DefaultParentShardPollIntervalMillis, - ShardSyncIntervalMillis: DefaultShardSyncIntervalMillis, - CleanupTerminatedShardsBeforeExpiry: DefaultCleanupLeasesUponShardsCompletion, - TaskBackoffTimeMillis: DefaultTaskBackoffTimeMillis, - ValidateSequenceNumberBeforeCheckpointing: DefaultValidateSequenceNumberBeforeCheckpointing, - ShutdownGraceMillis: DefaultShutdownGraceMillis, - MaxLeasesForWorker: DefaultMaxLeasesForWorker, - MaxLeasesToStealAtOneTime: DefaultMaxLeasesToStealAtOneTime, - InitialLeaseTableReadCapacity: DefaultInitialLeaseTableReadCapacity, + return &KinesisClientLibConfiguration { + ApplicationName: applicationName, + KinesisCredentials: kinesisCreds, + DynamoDBCredentials: dynamodbCreds, + TableName: applicationName, + EnhancedFanOutConsumerName: applicationName, + StreamName: streamName, + RegionName: regionName, + WorkerID: workerID, + InitialPositionInStream: DefaultInitialPositionInStream, + InitialPositionInStreamExtended: *newInitialPosition(DefaultInitialPositionInStream), + FailoverTimeMillis: DefaultFailoverTimeMillis, + LeaseRefreshPeriodMillis: DefaultLeaseRefreshPeriodMillis, + MaxRecords: DefaultMaxRecords, + IdleTimeBetweenReadsInMillis: DefaultIdleTimeBetweenReadsMillis, + CallProcessRecordsEvenForEmptyRecordList: DefaultDontCallProcessRecordsForEmptyRecordList, + ParentShardPollIntervalMillis: DefaultParentShardPollIntervalMillis, + ShardSyncIntervalMillis: DefaultShardSyncIntervalMillis, + CleanupTerminatedShardsBeforeExpiry: DefaultCleanupLeasesUponShardsCompletion, + TaskBackoffTimeMillis: DefaultTaskBackoffTimeMillis, + ValidateSequenceNumberBeforeCheckpointing: DefaultValidateSequenceNumberBeforeCheckpointing, + ShutdownGraceMillis: DefaultShutdownGraceMillis, + MaxLeasesForWorker: DefaultMaxLeasesForWorker, + MaxLeasesToStealAtOneTime: DefaultMaxLeasesToStealAtOneTime, + InitialLeaseTableReadCapacity: DefaultInitialLeaseTableReadCapacity, InitialLeaseTableWriteCapacity: DefaultInitialLeaseTableWriteCapacity, SkipShardSyncAtWorkerInitializationIfLeasesExist: DefaultSkipShardSyncAtStartupIfLeasesExist, EnableLeaseStealing: DefaultEnableLeaseStealing, diff --git a/clientlibrary/interfaces/inputs.go b/clientlibrary/interfaces/inputs.go index 385a3d2..2336af8 100644 --- a/clientlibrary/interfaces/inputs.go +++ b/clientlibrary/interfaces/inputs.go @@ -38,20 +38,20 @@ package interfaces import ( "time" - "github.com/aws/aws-sdk-go/aws" - ks "github.com/aws/aws-sdk-go/service/kinesis" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/kinesis/types" ) const ( - /** - * Indicates that the entire application is being shutdown, and if desired the record processor will be given a + /* + * REQUESTED Indicates that the entire application is being shutdown, and if desired the record processor will be given a * final chance to checkpoint. This state will not trigger a direct call to * {@link com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor#shutdown(ShutdownInput)}, but * instead depend on a different interface for backward compatibility. */ REQUESTED ShutdownReason = iota + 1 - /** + /* * Terminate processing for this RecordProcessor (resharding use case). * Indicates that the shard is closed and all records from the shard have been delivered to the application. * Applications SHOULD checkpoint their progress to indicate that they have successfully processed all records @@ -95,7 +95,7 @@ type ( CacheExitTime *time.Time // The records received from Kinesis. These records may have been de-aggregated if they were published by the KPL. - Records []*ks.Record + Records []types.Record // A checkpointer that the RecordProcessor can use to checkpoint its progress. Checkpointer IRecordProcessorCheckpointer diff --git a/clientlibrary/metrics/cloudwatch/cloudwatch.go b/clientlibrary/metrics/cloudwatch/cloudwatch.go index 2a1731d..144ce85 100644 --- a/clientlibrary/metrics/cloudwatch/cloudwatch.go +++ b/clientlibrary/metrics/cloudwatch/cloudwatch.go @@ -30,27 +30,28 @@ package cloudwatch import ( + "context" "sync" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/credentials" - "github.com/aws/aws-sdk-go/aws/session" - cwatch "github.com/aws/aws-sdk-go/service/cloudwatch" - "github.com/aws/aws-sdk-go/service/cloudwatch/cloudwatchiface" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/credentials" + + cwatch "github.com/aws/aws-sdk-go-v2/service/cloudwatch" + "github.com/aws/aws-sdk-go-v2/service/cloudwatch/types" "github.com/vmware/vmware-go-kcl/logger" ) -// Buffer metrics for at most this long before publishing to CloudWatch. -const DEFAULT_CLOUDWATCH_METRICS_BUFFER_DURATION = 10 * time.Second +// DefaultCloudwatchMetricsBufferDuration Buffer metrics for at most this long before publishing to CloudWatch. +const DefaultCloudwatchMetricsBufferDuration = 10 * time.Second type MonitoringService struct { appName string streamName string workerID string region string - credentials *credentials.Credentials + credentials *credentials.StaticCredentialsProvider logger logger.Logger // control how often to publish to CloudWatch @@ -58,7 +59,7 @@ type MonitoringService struct { stop *chan struct{} waitGroup *sync.WaitGroup - svc cloudwatchiface.CloudWatchAPI + svc *cwatch.Client shardMetrics *sync.Map } @@ -75,13 +76,13 @@ type cloudWatchMetrics struct { } // NewMonitoringService returns a Monitoring service publishing metrics to CloudWatch. -func NewMonitoringService(region string, creds *credentials.Credentials) *MonitoringService { - return NewMonitoringServiceWithOptions(region, creds, logger.GetDefaultLogger(), DEFAULT_CLOUDWATCH_METRICS_BUFFER_DURATION) +func NewMonitoringService(region string, creds *credentials.StaticCredentialsProvider) *MonitoringService { + return NewMonitoringServiceWithOptions(region, creds, logger.GetDefaultLogger(), DefaultCloudwatchMetricsBufferDuration) } // NewMonitoringServiceWithOptions returns a Monitoring service publishing metrics to // CloudWatch with the provided credentials, buffering duration and logger. -func NewMonitoringServiceWithOptions(region string, creds *credentials.Credentials, logger logger.Logger, bufferDur time.Duration) *MonitoringService { +func NewMonitoringServiceWithOptions(region string, creds *credentials.StaticCredentialsProvider, logger logger.Logger, bufferDur time.Duration) *MonitoringService { return &MonitoringService{ region: region, credentials: creds, @@ -95,15 +96,11 @@ func (cw *MonitoringService) Init(appName, streamName, workerID string) error { cw.streamName = streamName cw.workerID = workerID - cfg := &aws.Config{Region: aws.String(cw.region)} + cfg := &aws.Config{Region: cw.region} cfg.Credentials = cw.credentials - s, err := session.NewSession(cfg) - if err != nil { - cw.logger.Errorf("Error in creating session for cloudwatch. %+v", err) - return err - } - cw.svc = cwatch.New(s) - cw.shardMetrics = new(sync.Map) + + cw.svc = cwatch.NewFromConfig(*cfg) + cw.shardMetrics = &sync.Map{} stopChan := make(chan struct{}) cw.stop = &stopChan @@ -150,7 +147,7 @@ func (cw *MonitoringService) eventloop() { func (cw *MonitoringService) flushShard(shard string, metric *cloudWatchMetrics) bool { metric.Lock() - defaultDimensions := []*cwatch.Dimension{ + defaultDimensions := []types.Dimension{ { Name: aws.String("Shard"), Value: &shard, @@ -161,7 +158,7 @@ func (cw *MonitoringService) flushShard(shard string, metric *cloudWatchMetrics) }, } - leaseDimensions := []*cwatch.Dimension{ + leaseDimensions := []types.Dimension{ { Name: aws.String("Shard"), Value: &shard, @@ -177,44 +174,44 @@ func (cw *MonitoringService) flushShard(shard string, metric *cloudWatchMetrics) } metricTimestamp := time.Now() - data := []*cwatch.MetricDatum{ + data := []types.MetricDatum{ { Dimensions: defaultDimensions, MetricName: aws.String("RecordsProcessed"), - Unit: aws.String("Count"), + Unit: types.StandardUnitCount, Timestamp: &metricTimestamp, Value: aws.Float64(float64(metric.processedRecords)), }, { Dimensions: defaultDimensions, MetricName: aws.String("DataBytesProcessed"), - Unit: aws.String("Bytes"), + Unit: types.StandardUnitBytes, Timestamp: &metricTimestamp, Value: aws.Float64(float64(metric.processedBytes)), }, { Dimensions: leaseDimensions, MetricName: aws.String("RenewLease.Success"), - Unit: aws.String("Count"), + Unit: types.StandardUnitCount, Timestamp: &metricTimestamp, Value: aws.Float64(float64(metric.leaseRenewals)), }, { Dimensions: leaseDimensions, MetricName: aws.String("CurrentLeases"), - Unit: aws.String("Count"), + Unit: types.StandardUnitCount, Timestamp: &metricTimestamp, Value: aws.Float64(float64(metric.leasesHeld)), }, } if len(metric.behindLatestMillis) > 0 { - data = append(data, &cwatch.MetricDatum{ + data = append(data, types.MetricDatum{ Dimensions: defaultDimensions, MetricName: aws.String("MillisBehindLatest"), - Unit: aws.String("Milliseconds"), + Unit: types.StandardUnitMilliseconds, Timestamp: &metricTimestamp, - StatisticValues: &cwatch.StatisticSet{ + StatisticValues: &types.StatisticSet{ SampleCount: aws.Float64(float64(len(metric.behindLatestMillis))), Sum: sumFloat64(metric.behindLatestMillis), Maximum: maxFloat64(metric.behindLatestMillis), @@ -223,12 +220,12 @@ func (cw *MonitoringService) flushShard(shard string, metric *cloudWatchMetrics) } if len(metric.getRecordsTime) > 0 { - data = append(data, &cwatch.MetricDatum{ + data = append(data, types.MetricDatum{ Dimensions: defaultDimensions, MetricName: aws.String("KinesisDataFetcher.getRecords.Time"), - Unit: aws.String("Milliseconds"), + Unit: types.StandardUnitMilliseconds, Timestamp: &metricTimestamp, - StatisticValues: &cwatch.StatisticSet{ + StatisticValues: &types.StatisticSet{ SampleCount: aws.Float64(float64(len(metric.getRecordsTime))), Sum: sumFloat64(metric.getRecordsTime), Maximum: maxFloat64(metric.getRecordsTime), @@ -237,12 +234,12 @@ func (cw *MonitoringService) flushShard(shard string, metric *cloudWatchMetrics) } if len(metric.processRecordsTime) > 0 { - data = append(data, &cwatch.MetricDatum{ + data = append(data, types.MetricDatum{ Dimensions: defaultDimensions, MetricName: aws.String("RecordProcessor.processRecords.Time"), - Unit: aws.String("Milliseconds"), + Unit: types.StandardUnitMilliseconds, Timestamp: &metricTimestamp, - StatisticValues: &cwatch.StatisticSet{ + StatisticValues: &types.StatisticSet{ SampleCount: aws.Float64(float64(len(metric.processRecordsTime))), Sum: sumFloat64(metric.processRecordsTime), Maximum: maxFloat64(metric.processRecordsTime), @@ -251,7 +248,7 @@ func (cw *MonitoringService) flushShard(shard string, metric *cloudWatchMetrics) } // Publish metrics data to cloud watch - _, err := cw.svc.PutMetricData(&cwatch.PutMetricDataInput{ + _, err := cw.svc.PutMetricData(context.TODO(), &cwatch.PutMetricDataInput{ Namespace: aws.String(cw.appName), MetricData: data, }) diff --git a/clientlibrary/worker/common-shard-consumer.go b/clientlibrary/worker/common-shard-consumer.go index 416ac13..a638f5f 100644 --- a/clientlibrary/worker/common-shard-consumer.go +++ b/clientlibrary/worker/common-shard-consumer.go @@ -16,22 +16,24 @@ * WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE * SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + +// Package worker package worker import ( "sync" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/kinesis" - "github.com/aws/aws-sdk-go/service/kinesis/kinesisiface" - deagg "github.com/awslabs/kinesis-aggregation/go/deaggregator" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/kinesis" + "github.com/aws/aws-sdk-go-v2/service/kinesis/types" chk "github.com/vmware/vmware-go-kcl/clientlibrary/checkpoint" "github.com/vmware/vmware-go-kcl/clientlibrary/config" kcl "github.com/vmware/vmware-go-kcl/clientlibrary/interfaces" "github.com/vmware/vmware-go-kcl/clientlibrary/metrics" par "github.com/vmware/vmware-go-kcl/clientlibrary/partition" + deagg "github.com/vmware/vmware-go-kcl/internal/deaggregator" ) type shardConsumer interface { @@ -41,7 +43,7 @@ type shardConsumer interface { // commonShardConsumer implements common functionality for regular and enhanced fan-out consumers type commonShardConsumer struct { shard *par.ShardStatus - kc kinesisiface.KinesisAPI + kc *kinesis.Client checkpointer chk.Checkpointer recordProcessor kcl.IRecordProcessor kclConfig *config.KinesisClientLibConfiguration @@ -66,7 +68,7 @@ func (sc *commonShardConsumer) releaseLease() { // getStartingPosition gets kinesis stating position. // First try to fetch checkpoint. If checkpoint is not found use InitialPositionInStream -func (sc *commonShardConsumer) getStartingPosition() (*kinesis.StartingPosition, error) { +func (sc *commonShardConsumer) getStartingPosition() (*types.StartingPosition, error) { err := sc.checkpointer.FetchCheckpoint(sc.shard) if err != nil && err != chk.ErrSequenceIDNotFound { return nil, err @@ -75,24 +77,29 @@ func (sc *commonShardConsumer) getStartingPosition() (*kinesis.StartingPosition, checkpoint := sc.shard.GetCheckpoint() if checkpoint != "" { sc.kclConfig.Logger.Debugf("Start shard: %v at checkpoint: %v", sc.shard.ID, checkpoint) - return &kinesis.StartingPosition{ - Type: aws.String("AFTER_SEQUENCE_NUMBER"), + return &types.StartingPosition{ + Type: types.ShardIteratorTypeAfterSequenceNumber, SequenceNumber: &checkpoint, }, nil } shardIteratorType := config.InitalPositionInStreamToShardIteratorType(sc.kclConfig.InitialPositionInStream) - sc.kclConfig.Logger.Debugf("No checkpoint recorded for shard: %v, starting with: %v", sc.shard.ID, aws.StringValue(shardIteratorType)) - + sc.kclConfig.Logger.Debugf("No checkpoint recorded for shard: %v, starting with: %v", sc.shard.ID, aws.ToString(shardIteratorType)) if sc.kclConfig.InitialPositionInStream == config.AT_TIMESTAMP { - return &kinesis.StartingPosition{ - Type: shardIteratorType, + return &types.StartingPosition{ + Type: types.ShardIteratorTypeAtTimestamp, Timestamp: sc.kclConfig.InitialPositionInStreamExtended.Timestamp, }, nil } - return &kinesis.StartingPosition{ - Type: shardIteratorType, + if *shardIteratorType == "TRIM_HORIZON" { + return &types.StartingPosition{ + Type: types.ShardIteratorTypeTrimHorizon, + }, nil + } + + return &types.StartingPosition{ + Type: types.ShardIteratorTypeLatest, }, nil } @@ -121,7 +128,7 @@ func (sc *commonShardConsumer) waitOnParentShard() error { } } -func (sc *commonShardConsumer) processRecords(getRecordsStartTime time.Time, records []*kinesis.Record, millisBehindLatest *int64, recordCheckpointer kcl.IRecordProcessorCheckpointer) { +func (sc *commonShardConsumer) processRecords(getRecordsStartTime time.Time, records []types.Record, millisBehindLatest *int64, recordCheckpointer kcl.IRecordProcessorCheckpointer) { log := sc.kclConfig.Logger getRecordsTime := time.Since(getRecordsStartTime).Milliseconds() @@ -139,7 +146,7 @@ func (sc *commonShardConsumer) processRecords(getRecordsStartTime time.Time, rec input := &kcl.ProcessRecordsInput{ Records: dars, - MillisBehindLatest: aws.Int64Value(millisBehindLatest), + MillisBehindLatest: *millisBehindLatest, Checkpointer: recordCheckpointer, } diff --git a/clientlibrary/worker/fan-out-shard-consumer.go b/clientlibrary/worker/fan-out-shard-consumer.go index ba4484c..76827f3 100644 --- a/clientlibrary/worker/fan-out-shard-consumer.go +++ b/clientlibrary/worker/fan-out-shard-consumer.go @@ -16,14 +16,18 @@ * WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE * SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + +// Package worker package worker import ( + "context" "errors" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/kinesis" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/kinesis" + "github.com/aws/aws-sdk-go-v2/service/kinesis/types" chk "github.com/vmware/vmware-go-kcl/clientlibrary/checkpoint" kcl "github.com/vmware/vmware-go-kcl/clientlibrary/interfaces" @@ -61,11 +65,11 @@ func (sc *FanOutShardConsumer) getRecords() error { return err } defer func() { - if shardSub == nil || shardSub.EventStream == nil { + if shardSub == nil || shardSub.GetStream() == nil { log.Debugf("Nothing to close, EventStream is nil") return } - err = shardSub.EventStream.Close() + err = shardSub.GetStream().Close() if err != nil { log.Errorf("Unable to close event stream for %s: %v", sc.shard.ID, err) } @@ -99,7 +103,7 @@ func (sc *FanOutShardConsumer) getRecords() error { return err } refreshLeaseTimer = time.After(time.Until(sc.shard.LeaseTimeout.Add(-time.Duration(sc.kclConfig.LeaseRefreshPeriodMillis) * time.Millisecond))) - case event, ok := <-shardSub.EventStream.Events(): + case event, ok := <-shardSub.GetStream().Events(): if !ok { // need to resubscribe to shard log.Debugf("Event stream ended, refreshing subscription on shard: %s for worker: %s", sc.shard.ID, sc.consumerID) @@ -113,13 +117,13 @@ func (sc *FanOutShardConsumer) getRecords() error { } continue } - subEvent, ok := event.(*kinesis.SubscribeToShardEvent) + subEvent, ok := event.(*types.SubscribeToShardEventStreamMemberSubscribeToShardEvent) if !ok { log.Errorf("Received unexpected event type: %T", event) continue } - continuationSequenceNumber = subEvent.ContinuationSequenceNumber - sc.processRecords(getRecordsStartTime, subEvent.Records, subEvent.MillisBehindLatest, recordCheckpointer) + continuationSequenceNumber = subEvent.Value.ContinuationSequenceNumber + sc.processRecords(getRecordsStartTime, subEvent.Value.Records, subEvent.Value.MillisBehindLatest, recordCheckpointer) // The shard has been closed, so no new records can be read from it if continuationSequenceNumber == nil { @@ -138,7 +142,7 @@ func (sc *FanOutShardConsumer) subscribeToShard() (*kinesis.SubscribeToShardOutp return nil, err } - return sc.kc.SubscribeToShard(&kinesis.SubscribeToShardInput{ + return sc.kc.SubscribeToShard(context.TODO(), &kinesis.SubscribeToShardInput{ ConsumerARN: &sc.consumerARN, ShardId: &sc.shard.ID, StartingPosition: startPosition, @@ -146,16 +150,16 @@ func (sc *FanOutShardConsumer) subscribeToShard() (*kinesis.SubscribeToShardOutp } func (sc *FanOutShardConsumer) resubscribe(shardSub *kinesis.SubscribeToShardOutput, continuationSequence *string) (*kinesis.SubscribeToShardOutput, error) { - err := shardSub.EventStream.Close() + err := shardSub.GetStream().Close() if err != nil { sc.kclConfig.Logger.Errorf("Unable to close event stream for %s: %v", sc.shard.ID, err) return nil, err } - startPosition := &kinesis.StartingPosition{ - Type: aws.String("AFTER_SEQUENCE_NUMBER"), + startPosition := &types.StartingPosition{ + Type: types.ShardIteratorTypeAfterSequenceNumber, SequenceNumber: continuationSequence, } - shardSub, err = sc.kc.SubscribeToShard(&kinesis.SubscribeToShardInput{ + shardSub, err = sc.kc.SubscribeToShard(context.TODO(), &kinesis.SubscribeToShardInput{ ConsumerARN: &sc.consumerARN, ShardId: &sc.shard.ID, StartingPosition: startPosition, diff --git a/clientlibrary/worker/polling-shard-consumer.go b/clientlibrary/worker/polling-shard-consumer.go index 90371b0..f0a1d9e 100644 --- a/clientlibrary/worker/polling-shard-consumer.go +++ b/clientlibrary/worker/polling-shard-consumer.go @@ -16,6 +16,8 @@ * WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE * SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + +// Package worker // The implementation is derived from https://github.com/patrobinson/gokini // // Copyright 2018 Patrick robinson @@ -28,17 +30,18 @@ package worker import ( + "context" "errors" "math" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/kinesis" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/kinesis" + "github.com/aws/aws-sdk-go-v2/service/kinesis/types" chk "github.com/vmware/vmware-go-kcl/clientlibrary/checkpoint" kcl "github.com/vmware/vmware-go-kcl/clientlibrary/interfaces" "github.com/vmware/vmware-go-kcl/clientlibrary/metrics" - "github.com/vmware/vmware-go-kcl/clientlibrary/utils" ) // PollingShardConsumer is responsible for polling data records from a (specified) shard. @@ -56,6 +59,7 @@ func (sc *PollingShardConsumer) getShardIterator() (*string, error) { if err != nil { return nil, err } + shardIterArgs := &kinesis.GetShardIteratorInput{ ShardId: &sc.shard.ID, ShardIteratorType: startPosition.Type, @@ -63,14 +67,16 @@ func (sc *PollingShardConsumer) getShardIterator() (*string, error) { Timestamp: startPosition.Timestamp, StreamName: &sc.streamName, } - iterResp, err := sc.kc.GetShardIterator(shardIterArgs) + + iterResp, err := sc.kc.GetShardIterator(context.TODO(), shardIterArgs) if err != nil { return nil, err } + return iterResp.ShardIterator, nil } -// getRecords continously poll one shard for data record +// getRecords continuously poll one shard for data record // Precondition: it currently has the lease on the shard. func (sc *PollingShardConsumer) getRecords() error { defer sc.releaseLease() @@ -120,15 +126,19 @@ func (sc *PollingShardConsumer) getRecords() error { getRecordsStartTime := time.Now() - log.Debugf("Trying to read %d record from iterator: %v", sc.kclConfig.MaxRecords, aws.StringValue(shardIterator)) + log.Debugf("Trying to read %d record from iterator: %v", sc.kclConfig.MaxRecords, aws.ToString(shardIterator)) getRecordsArgs := &kinesis.GetRecordsInput{ - Limit: aws.Int64(int64(sc.kclConfig.MaxRecords)), + Limit: aws.Int32(int32(sc.kclConfig.MaxRecords)), ShardIterator: shardIterator, } + // Get records from stream and retry as needed - getResp, err := sc.kc.GetRecords(getRecordsArgs) + getResp, err := sc.kc.GetRecords(context.TODO(), getRecordsArgs) if err != nil { - if utils.AWSErrCode(err) == kinesis.ErrCodeProvisionedThroughputExceededException || utils.AWSErrCode(err) == kinesis.ErrCodeKMSThrottlingException { + //aws-sdk-go-v2 https://github.com/aws/aws-sdk-go-v2/blob/main/CHANGELOG.md#error-handling + var throughputExceededErr *types.ProvisionedThroughputExceededException + var kmsThrottlingErr *types.KMSThrottlingException + if errors.As(err, &throughputExceededErr) || errors.As(err, &kmsThrottlingErr) { log.Errorf("Error getting records from shard %v: %+v", sc.shard.ID, err) retriedErrors++ // exponential backoff @@ -156,7 +166,7 @@ func (sc *PollingShardConsumer) getRecords() error { // Idle between each read, the user is responsible for checkpoint the progress // This value is only used when no records are returned; if records are returned, it should immediately // retrieve the next set of records. - if len(getResp.Records) == 0 && aws.Int64Value(getResp.MillisBehindLatest) < int64(sc.kclConfig.IdleTimeBetweenReadsInMillis) { + if len(getResp.Records) == 0 && aws.ToInt64(getResp.MillisBehindLatest) < int64(sc.kclConfig.IdleTimeBetweenReadsInMillis) { time.Sleep(time.Duration(sc.kclConfig.IdleTimeBetweenReadsInMillis) * time.Millisecond) } diff --git a/clientlibrary/worker/record-processor-checkpointer.go b/clientlibrary/worker/record-processor-checkpointer.go index cf96ea9..c89dc4a 100644 --- a/clientlibrary/worker/record-processor-checkpointer.go +++ b/clientlibrary/worker/record-processor-checkpointer.go @@ -21,6 +21,8 @@ package worker import ( + "github.com/aws/aws-sdk-go-v2/aws" + chk "github.com/vmware/vmware-go-kcl/clientlibrary/checkpoint" kcl "github.com/vmware/vmware-go-kcl/clientlibrary/interfaces" par "github.com/vmware/vmware-go-kcl/clientlibrary/partition" @@ -71,13 +73,12 @@ func (rc *RecordProcessorCheckpointer) Checkpoint(sequenceNumber *string) error if sequenceNumber == nil { rc.shard.SetCheckpoint(chk.ShardEnd) } else { - rc.shard.SetCheckpoint(aws.StringValue(sequenceNumber)) + rc.shard.SetCheckpoint(aws.ToString(sequenceNumber)) } return rc.checkpoint.CheckpointSequence(rc.shard) } -func (rc *RecordProcessorCheckpointer) PrepareCheckpoint(sequenceNumber *string) (kcl.IPreparedCheckpointer, error) { +func (rc *RecordProcessorCheckpointer) PrepareCheckpoint(_ *string) (kcl.IPreparedCheckpointer, error) { return &PreparedCheckpointer{}, nil - } diff --git a/clientlibrary/worker/worker-fan-out.go b/clientlibrary/worker/worker-fan-out.go index 0725671..a7943b3 100644 --- a/clientlibrary/worker/worker-fan-out.go +++ b/clientlibrary/worker/worker-fan-out.go @@ -16,16 +16,19 @@ * WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE * SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + +// Package worker package worker import ( + "context" + "errors" "fmt" "math" "time" - "github.com/aws/aws-sdk-go/service/kinesis" - - "github.com/vmware/vmware-go-kcl/clientlibrary/utils" + "github.com/aws/aws-sdk-go-v2/service/kinesis" + "github.com/aws/aws-sdk-go-v2/service/kinesis/types" ) // fetchConsumerARNWithRetry tries to fetch consumer ARN. Retries 10 times with exponential backoff in case of an error @@ -50,27 +53,34 @@ func (w *Worker) fetchConsumerARNWithRetry() (string, error) { func (w *Worker) fetchConsumerARN() (string, error) { log := w.kclConfig.Logger log.Debugf("Fetching stream consumer ARN") - streamDescription, err := w.kc.DescribeStream(&kinesis.DescribeStreamInput{ + + streamDescription, err := w.kc.DescribeStream(context.TODO(), &kinesis.DescribeStreamInput{ StreamName: &w.kclConfig.StreamName, }) + if err != nil { log.Errorf("Could not describe stream: %v", err) return "", err } - streamConsumerDescription, err := w.kc.DescribeStreamConsumer(&kinesis.DescribeStreamConsumerInput{ + + streamConsumerDescription, err := w.kc.DescribeStreamConsumer(context.TODO(), &kinesis.DescribeStreamConsumerInput{ ConsumerName: &w.kclConfig.EnhancedFanOutConsumerName, StreamARN: streamDescription.StreamDescription.StreamARN, }) + if err == nil { - log.Infof("Enhanced fan-out consumer found, consumer status: %s", *streamConsumerDescription.ConsumerDescription.ConsumerStatus) - if *streamConsumerDescription.ConsumerDescription.ConsumerStatus != kinesis.ConsumerStatusActive { - return "", fmt.Errorf("consumer is not in active status yet, current status: %s", *streamConsumerDescription.ConsumerDescription.ConsumerStatus) + log.Infof("Enhanced fan-out consumer found, consumer status: %s", streamConsumerDescription.ConsumerDescription.ConsumerStatus) + if streamConsumerDescription.ConsumerDescription.ConsumerStatus != types.ConsumerStatusActive { + return "", fmt.Errorf("consumer is not in active status yet, current status: %s", streamConsumerDescription.ConsumerDescription.ConsumerStatus) } return *streamConsumerDescription.ConsumerDescription.ConsumerARN, nil } - if utils.AWSErrCode(err) == kinesis.ErrCodeResourceNotFoundException { + + //aws-sdk-go-v2 https://github.com/aws/aws-sdk-go-v2/blob/main/CHANGELOG.md#error-handling + var notFoundErr *types.ResourceNotFoundException + if errors.As(err, ¬FoundErr) { log.Infof("Enhanced fan-out consumer not found, registering new consumer with name: %s", w.kclConfig.EnhancedFanOutConsumerName) - out, err := w.kc.RegisterStreamConsumer(&kinesis.RegisterStreamConsumerInput{ + out, err := w.kc.RegisterStreamConsumer(context.TODO(), &kinesis.RegisterStreamConsumerInput{ ConsumerName: &w.kclConfig.EnhancedFanOutConsumerName, StreamARN: streamDescription.StreamDescription.StreamARN, }) @@ -78,11 +88,13 @@ func (w *Worker) fetchConsumerARN() (string, error) { log.Errorf("Could not register enhanced fan-out consumer: %v", err) return "", err } - if *out.Consumer.ConsumerStatus != kinesis.ConsumerStatusActive { - return "", fmt.Errorf("consumer is not in active status yet, current status: %s", *out.Consumer.ConsumerStatus) + if out.Consumer.ConsumerStatus != types.ConsumerStatusActive { + return "", fmt.Errorf("consumer is not in active status yet, current status: %s", out.Consumer.ConsumerStatus) } return *out.Consumer.ConsumerARN, nil } - log.Errorf("Could not describe stream consumer: %v", err) + + log.Errorf("Could not describe stream consumer: %v", err) //%w should we unwrap the underlying error? + return "", err } diff --git a/clientlibrary/worker/worker.go b/clientlibrary/worker/worker.go index c5b39f4..01db8f5 100644 --- a/clientlibrary/worker/worker.go +++ b/clientlibrary/worker/worker.go @@ -30,16 +30,18 @@ package worker import ( + "context" "crypto/rand" "errors" "math/big" "sync" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/session" - "github.com/aws/aws-sdk-go/service/kinesis" - "github.com/aws/aws-sdk-go/service/kinesis/kinesisiface" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/aws/retry" + awsConfig "github.com/aws/aws-sdk-go-v2/config" + "github.com/aws/aws-sdk-go-v2/credentials" + "github.com/aws/aws-sdk-go-v2/service/kinesis" chk "github.com/vmware/vmware-go-kcl/clientlibrary/checkpoint" "github.com/vmware/vmware-go-kcl/clientlibrary/config" @@ -59,7 +61,7 @@ type Worker struct { processorFactory kcl.IRecordProcessorFactory kclConfig *config.KinesisClientLibConfiguration - kc kinesisiface.KinesisAPI + kc *kinesis.Client checkpointer chk.Checkpointer mService metrics.MonitoringService @@ -94,7 +96,7 @@ func NewWorker(factory kcl.IRecordProcessorFactory, kclConfig *config.KinesisCli } // WithKinesis is used to provide Kinesis service for either custom implementation or unit testing. -func (w *Worker) WithKinesis(svc kinesisiface.KinesisAPI) *Worker { +func (w *Worker) WithKinesis(svc *kinesis.Client) *Worker { w.kc = svc return w } @@ -153,22 +155,38 @@ func (w *Worker) initialize() error { log := w.kclConfig.Logger log.Infof("Worker initialization in progress...") - // Create default Kinesis session + // Create default Kinesis client if w.kc == nil { // create session for Kinesis - log.Infof("Creating Kinesis session") + log.Infof("Creating Kinesis client") - s, err := session.NewSession(&aws.Config{ - Region: aws.String(w.regionName), - Endpoint: &w.kclConfig.KinesisEndpoint, - Credentials: w.kclConfig.KinesisCredentials, + resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) { + return aws.Endpoint{ + PartitionID: "aws", + URL: w.kclConfig.KinesisEndpoint, + SigningRegion: w.regionName, + }, nil }) + cfg, err := awsConfig.LoadDefaultConfig( + context.TODO(), + awsConfig.WithRegion(w.regionName), + awsConfig.WithCredentialsProvider( + credentials.NewStaticCredentialsProvider( + w.kclConfig.KinesisCredentials.Value.AccessKeyID, + w.kclConfig.KinesisCredentials.Value.SecretAccessKey, + w.kclConfig.KinesisCredentials.Value.SessionToken)), + awsConfig.WithEndpointResolver(resolver), + awsConfig.WithRetryer(func() aws.Retryer { + return retry.AddWithMaxBackoffDelay(retry.NewStandard(), retry.DefaultMaxBackoff) + }), + ) + if err != nil { // no need to move forward - log.Fatalf("Failed in getting Kinesis session for creating Worker: %+v", err) + log.Fatalf("Failed in loading Kinesis default config for creating Worker: %+v", err) } - w.kc = kinesis.New(s) + w.kc = kinesis.NewFromConfig(cfg) } else { log.Infof("Use custom Kinesis service.") } @@ -460,7 +478,7 @@ func (w *Worker) getShardIDs(nextToken string, shardInfo map[string]bool) error args.StreamName = aws.String(w.streamName) } - listShards, err := w.kc.ListShards(args) + listShards, err := w.kc.ListShards(context.TODO(), args) if err != nil { log.Errorf("Error in ListShards: %s Error: %+v Request: %s", w.streamName, err, args) return err @@ -475,16 +493,16 @@ func (w *Worker) getShardIDs(nextToken string, shardInfo map[string]bool) error log.Infof("Found new shard with id %s", *s.ShardId) w.shardStatus[*s.ShardId] = &par.ShardStatus{ ID: *s.ShardId, - ParentShardId: aws.StringValue(s.ParentShardId), + ParentShardId: aws.ToString(s.ParentShardId), Mux: &sync.RWMutex{}, - StartingSequenceNumber: aws.StringValue(s.SequenceNumberRange.StartingSequenceNumber), - EndingSequenceNumber: aws.StringValue(s.SequenceNumberRange.EndingSequenceNumber), + StartingSequenceNumber: aws.ToString(s.SequenceNumberRange.StartingSequenceNumber), + EndingSequenceNumber: aws.ToString(s.SequenceNumberRange.EndingSequenceNumber), } } } if listShards.NextToken != nil { - err := w.getShardIDs(aws.StringValue(listShards.NextToken), shardInfo) + err := w.getShardIDs(aws.ToString(listShards.NextToken), shardInfo) if err != nil { log.Errorf("Error in ListShards: %s Error: %+v Request: %s", w.streamName, err, args) return err diff --git a/go.mod b/go.mod index 264834c..1f6152c 100644 --- a/go.mod +++ b/go.mod @@ -3,13 +3,18 @@ module github.com/vmware/vmware-go-kcl go 1.17 require ( - github.com/aws/aws-sdk-go v1.41.7 - github.com/awslabs/kinesis-aggregation/go v0.0.0-20210630091500-54e17340d32f + github.com/aws/aws-sdk-go-v2 v1.11.0 + github.com/aws/aws-sdk-go-v2/config v1.10.0 + github.com/aws/aws-sdk-go-v2/credentials v1.6.0 + github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.4.0 + github.com/aws/aws-sdk-go-v2/service/cloudwatch v1.10.0 + github.com/aws/aws-sdk-go-v2/service/dynamodb v1.7.0 + github.com/aws/aws-sdk-go-v2/service/kinesis v1.8.0 github.com/golang/protobuf v1.5.2 github.com/google/uuid v1.3.0 github.com/prometheus/client_golang v1.11.0 github.com/prometheus/common v0.32.1 - github.com/rs/zerolog v1.25.0 + github.com/rs/zerolog v1.26.0 github.com/sirupsen/logrus v1.8.1 github.com/stretchr/testify v1.7.0 go.uber.org/zap v1.19.1 @@ -18,6 +23,19 @@ require ( require ( github.com/BurntSushi/toml v0.4.1 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.0.0 // indirect + github.com/aws/aws-sdk-go-v2/feature/dynamodb/expression v1.3.0 + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.8.0 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.0 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.0.0 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.3.0 // indirect + github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.6.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.5.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.3.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.5.0 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.6.0 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.9.0 // indirect + github.com/aws/smithy-go v1.9.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.1.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect @@ -26,9 +44,10 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/procfs v0.7.3 // indirect + github.com/stretchr/objx v0.3.0 // indirect go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.7.0 // indirect - golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac // indirect + golang.org/x/sys v0.0.0-20211106132015-ebca88c72f68 // indirect google.golang.org/protobuf v1.27.1 // indirect gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect ) diff --git a/go.sum b/go.sum index 8fc9eba..22eed71 100644 --- a/go.sum +++ b/go.sum @@ -12,37 +12,150 @@ cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bP cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= +cloud.google.com/go v0.65.0 h1:Dg9iHVQfrhq82rUNu9ZxUDrJLaxFUe/HlCVaLyRruq8= cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= +cloud.google.com/go/bigquery v1.8.0 h1:PQcPefKFdaIzjQFbiyOgAqyx8q5djaE7x9Sqe712DPA= cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0 h1:/May9ojXjRkPBNVrq+oWLqmWCkr4OU5uRY29bu0mRyQ= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/pubsub v1.3.1 h1:ukjixP1wl0LpnZ6LWtZJ0mX5tBmjp1f8Sqer8Z2OMUU= cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= +cloud.google.com/go/storage v1.10.0 h1:STgFzyU5/8miMl0//zKh2aQeTyeaUH3WN9bSUiJ09bA= cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9 h1:VpgP7xuJadIUuKccphEpTJnWhS2jkQyMt6Y7pJCD7fY= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/toml v0.4.1 h1:GaI7EiDXDRfa8VshkTj7Fym7ha+y8/XxIgD2okUIjLw= github.com/BurntSushi/toml v0.4.1/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802 h1:1BDTz0u9nC3//pOCMdNH+CiXJVYJh5UQNCOBG7jbELc= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d h1:UQZhZ2O0vMHr2cI+DC1Mbh0TJxzA3RcLoMsFw+aXw7E= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= +github.com/aws/aws-sdk-go v1.19.48 h1:YhKzuc9xggUt8jNDc5CmIBeB8GmGtazzq0aCXO4sj6w= github.com/aws/aws-sdk-go v1.19.48/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.41.7 h1:vlpR8Cky3ZxUVNINgeRZS6N0p6zmFvu/ZqRRwrTI25U= github.com/aws/aws-sdk-go v1.41.7/go.mod h1:585smgzpB/KqRA+K3y/NL/oYRqQvpNJYvLm+LY1U59Q= +github.com/aws/aws-sdk-go-v2 v1.8.1/go.mod h1:xEFuWz+3TYdlPRuo+CqATbeDWIWyaT5uAPwPaWtgse0= +github.com/aws/aws-sdk-go-v2 v1.9.0/go.mod h1:cK/D0BBs0b/oWPIcX/Z/obahJK1TT7IPVjy53i/mX/4= +github.com/aws/aws-sdk-go-v2 v1.10.0 h1:+dCJ5W2HiZNa4UtaIc5ljKNulm0dK0vS5dxb5LdDOAA= +github.com/aws/aws-sdk-go-v2 v1.10.0/go.mod h1:U/EyyVvKtzmFeQQcca7eBotKdlpcP2zzU6bXBYcf7CE= +github.com/aws/aws-sdk-go-v2 v1.11.0 h1:HxyD62DyNhCfiFGUHqJ/xITD6rAjJ7Dm/2nLxLmO4Ag= +github.com/aws/aws-sdk-go-v2 v1.11.0/go.mod h1:SQfA+m2ltnu1cA0soUkj4dRSsmITiVQUJvBIZjzfPyQ= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.0.0 h1:yVUAwvJC/0WNPbyl0nA3j1L6CW1CN8wBubCRqtG7JLI= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.0.0/go.mod h1:Xn6sxgRuIDflLRJFj5Ev7UxABIkNbccFPV/p8itDReM= +github.com/aws/aws-sdk-go-v2/config v1.6.1 h1:qrZINaORyr78syO1zfD4l7r4tZjy0Z1l0sy4jiysyOM= +github.com/aws/aws-sdk-go-v2/config v1.6.1/go.mod h1:t/y3UPu0XEDy0cEw6mvygaBQaPzWiYAxfP2SzgtvclA= +github.com/aws/aws-sdk-go-v2/config v1.9.0 h1:SkREVSwi+J8MSdjhJ96jijZm5ZDNleI0E4hHCNivh7s= +github.com/aws/aws-sdk-go-v2/config v1.9.0/go.mod h1:qhK5NNSgo9/nOSMu3HyE60WHXZTWTHTgd5qtIF44vOQ= +github.com/aws/aws-sdk-go-v2/config v1.10.0 h1:4i+/7DmCQCAls5Z61giur0LOPZ3PXFwnSIw7hRamzws= +github.com/aws/aws-sdk-go-v2/config v1.10.0/go.mod h1:xuqoV5etD3N3B8Ts9je4ijgAv6mb+6NiOPFMUhwRcjA= +github.com/aws/aws-sdk-go-v2/credentials v1.3.3 h1:A13QPatmUl41SqUfnuT3V0E3XiNGL6qNTOINbE8cZL4= +github.com/aws/aws-sdk-go-v2/credentials v1.3.3/go.mod h1:oVieKMT3m9BSfqhOfuQ+E0j/yN84ZAJ7Qv8Sfume/ak= +github.com/aws/aws-sdk-go-v2/credentials v1.5.0 h1:r6470olsn2qyOe2aLzK6q+wfO3dzNcMujRT3gqBgBB8= +github.com/aws/aws-sdk-go-v2/credentials v1.5.0/go.mod h1:kvqTkpzQmzri9PbsiTY+LvwFzM0gY19emlAWwBOJMb0= +github.com/aws/aws-sdk-go-v2/credentials v1.6.0 h1:L3O6osQTlzLKRmiTphw2QJuD21EFapWCX4IipiRJhAE= +github.com/aws/aws-sdk-go-v2/credentials v1.6.0/go.mod h1:rQkYdQPDXRrvPLeEuCNwSgtwMzBo9eDGWlTNC69Sh/0= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.2.0 h1:8kvinmbIDObqsWegKP0JjeanYPiA4GUVpAtciNWE+jw= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.2.0/go.mod h1:UVFtSYSWCHj2+brBLDHUdlJXmz8LxUpZhA+Ewypc+xQ= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.3.0 h1:jEWmr4fcoAdoDo34DKMED/lEgPyyGE6/Xhwbgs6+NS8= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.3.0/go.mod h1:YjXozu6rHksfG22T5ZZASTrFOLzI0AoyuEC+GU9I3Lw= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.4.0 h1:J8Zgr+z0RjxidWB6vjX6sEB8TU/y6ELWoYhNoJ99d+M= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.4.0/go.mod h1:gWzcyoZ5LNkx1Xhluc25HU9eWIdcwiaymHuJnwO6ELs= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/expression v1.3.0 h1:Nm2gF15BCti2SRfE/G6rS7KbTD8mQTVIwGFjMZIlie0= +github.com/aws/aws-sdk-go-v2/feature/dynamodb/expression v1.3.0/go.mod h1:GQd3X3up0vqgHmt2jca0vyM7rbZj1KkJBDHlb6Oc1Eg= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.4.1 h1:rc+fRGvlKbeSd9IFhFS1KWBs0XjTkq0CfK5xqyLgIp0= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.4.1/go.mod h1:+GTydg3uHmVlQdkRoetz6VHKbOMEYof70m19IpMLifc= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.7.0 h1:FKaqk7geL3oIqSwGJt5SWUKj8uJ+qLZNqlBuqq6sFyA= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.7.0/go.mod h1:KqEkRkxm/+1Pd/rENRNbQpfblDBYeg5HDSqjB6ks8hA= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.8.0 h1:OpZjuUy8Jt3CA1WgJgBC5Bz+uOjE5Ppx4NFTRaooUuA= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.8.0/go.mod h1:5E1J3/TTYy6z909QNR0QnXGBpfESYGDqd3O0zqONghU= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.0.4 h1:IM9b6hlCcVFJFydPoyphs/t7YrHfqKy7T4/7AG5Eprs= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.0.4/go.mod h1:W5gGbtNXFpF9/ssYZTaItzG/B+j0bjTnwStiCP2AtWU= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.0.7 h1:/0GQVY8J25hww4J9a+rYKDr9ryGh2KdIdR8YHBP54h0= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.0.7/go.mod h1:QXoZAXmBEHeMIFiBr3XumpTyoNTXTQbqPV+qaGX7gfY= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.0 h1:zY8cNmbBXt3pzjgWgdIbzpQ6qxoCwt+Nx9JbrAf2mbY= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.0/go.mod h1:NO3Q5ZTTQtO2xIg2+xTXYDiT7knSejfeDm7WGDaOo0U= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.0.0 h1:Z3aR/OXBnkYK9zXkNkfitHX6SmUBzSsx8VMHbH4Lvhw= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.0.0/go.mod h1:anlUzBoEWglcUxUQwZA7HQOEVEnQALVZsizAapB2hq8= +github.com/aws/aws-sdk-go-v2/internal/ini v1.2.1 h1:IkqRRUZTKaS16P2vpX+FNc2jq3JWa3c478gykQp4ow4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.2.1/go.mod h1:Pv3WenDjI0v2Jl7UaMFIIbPOBbhn33RmmAmGgkXDoqY= +github.com/aws/aws-sdk-go-v2/internal/ini v1.2.5 h1:zPxLGWALExNepElO0gYgoqsbqTlt4ZCrhZ7XlfJ+Qlw= +github.com/aws/aws-sdk-go-v2/internal/ini v1.2.5/go.mod h1:6ZBTuDmvpCOD4Sf1i2/I3PgftlEcDGgvi8ocq64oQEg= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.0 h1:c10Z7fWxtJCoyc8rv06jdh9xrKnu7bAJiRaKWvTb2mU= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.0/go.mod h1:6oXGy4GLpypD3uCh8wcqztigGgmhLToMfjavgh+VySg= +github.com/aws/aws-sdk-go-v2/service/cloudwatch v1.9.0/go.mod h1:iX4tSMY8NP1mzU2PMS6arLyB/Yufz2LxCkn9DsgiWEI= +github.com/aws/aws-sdk-go-v2/service/cloudwatch v1.10.0 h1:MNNV0fi3J5Lxxhx8iDlKdRZJrtBv/0FyganA3nBYe8Q= +github.com/aws/aws-sdk-go-v2/service/cloudwatch v1.10.0/go.mod h1:Oiwhs3Fo9amYOGsJggWBPU6bwa/u0xVpEdOS5HlouPg= +github.com/aws/aws-sdk-go-v2/service/dynamodb v1.5.0 h1:SGwKUQaJudQQZE72dDQlL2FGuHNAEK1CyqKLTjh6mqE= +github.com/aws/aws-sdk-go-v2/service/dynamodb v1.5.0/go.mod h1:XY5YhCS9SLul3JSQ08XG/nfxXxrkh6RR21XPq/J//NY= +github.com/aws/aws-sdk-go-v2/service/dynamodb v1.6.0 h1:HDp8hUQlGU5fgNoNDp0BOthk57AuTXMTaAK1mb9c27I= +github.com/aws/aws-sdk-go-v2/service/dynamodb v1.6.0/go.mod h1:t8pYXJHxfOe/088CcNeuqQbucpq9SwO1yjheCieDDnI= +github.com/aws/aws-sdk-go-v2/service/dynamodb v1.7.0 h1:S3X6RWl0TfMxNXsIzz8r3Y6YVA1HWGSx6M345Q3mQ+I= +github.com/aws/aws-sdk-go-v2/service/dynamodb v1.7.0/go.mod h1:Hh0zJ3419ET9xQBeR+y0lHIkObJwAKPbzV9nTZ0yrJ0= +github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.4.0 h1:QbFWJr2SAyVYvyoOHvJU6sCGLnqNT94ZbWElJMEI1JY= +github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.4.0/go.mod h1:bYsEP8w5YnbYyrx/Zi5hy4hTwRRQISSJS3RWrsGRijg= +github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.5.0 h1:At4HitvrEFdSA5rNS1KHA65BYizq2p+gLtASYtoAH2A= +github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.5.0/go.mod h1:9u/PDp7T3XzjGA8XmYJcffjqPJmXeofDXHUyHqp2lYc= +github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.6.0 h1:Z893Baw1+7PfK+KtYgrHu+V2n/Ae9S0jG1dZGe4WQ7o= +github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.6.0/go.mod h1:PmJdIbYf6UjqnAJwZPi6CNG8JHXdzc/Y0Y8bWfPy0Yw= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 h1:gceOysEWNNwLd6cki65IMBZ4WAM0MwgBQq2n7kejoT8= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0/go.mod h1:v8ygadNyATSm6elwJ/4gzJwcFhri9RqS8skgHKiwXPU= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.4.0 h1:EtQ6hVAgNsWTiO+u9e+ziaEYyOAlEkAwLskpL40U6pQ= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.4.0/go.mod h1:vEkJTjJ8vnv0uWy2tAp7DSydWFpudMGWPQ2SFucoN1k= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.5.0 h1:lPLbw4Gn59uoKqvOfSnkJr54XWk5Ak1NK20ZEiSWb3U= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.5.0/go.mod h1:80NaCIH9YU3rzTTs/J/ECATjXuRqzo/wB6ukO6MZ0XY= +github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.1.0 h1:QCPbsMPMcM4iGbui5SH6O4uxvZffPoBJ4CIGX7dU0l4= +github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.1.0/go.mod h1:enkU5tq2HoXY+ZMiQprgF3Q83T3PbO77E83yXXzRZWE= +github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.2.0 h1:uxy31f/H1bkUV2aircA9hTQT8s093u1eOeErsOXIY90= +github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.2.0/go.mod h1:wLLzEoPune3u08rkvNBm3BprebkWRmmCkMtTeujM3Fs= +github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.3.0 h1:A2aUh9d38A2ECh76ahOQUdpJFe+Jhjk8qrfV+YbNYGY= +github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.3.0/go.mod h1:5h2rxfLN22pLTQ1ZoOza87rp2SnN/9UDYdYBQRmIrsE= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.3 h1:VxFCgxsqWe7OThOwJ5IpFX3xrObtuIH9Hg/NW7oot1Y= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.3/go.mod h1:7gcsONBmFoCcKrAqrm95trrMd2+C/ReYKP7Vfu8yHHA= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.4.0 h1:/T5wKsw/po118HEDvnSE8YU7TESxvZbYM2rnn+Oi7Kk= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.4.0/go.mod h1:X5/JuOxPLU/ogICgDTtnpfaQzdQJO0yKDcpoxWLLJ8Y= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.5.0 h1:qGZWS/WgiFY+Zgad2u0gwBHpJxz6Ne401JE7iQI1nKs= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.5.0/go.mod h1:Mq6AEc+oEjCUlBuLiK5YwW4shSOAKCQ3tXN0sQeYoBA= +github.com/aws/aws-sdk-go-v2/service/kinesis v1.7.0 h1:BR1dH17nltcO/bRr7sW+BTOY1OekCf3KadPBTHWPIY4= +github.com/aws/aws-sdk-go-v2/service/kinesis v1.7.0/go.mod h1:IKVo9L4q2SkAfCZik0P9fdCZWppIE06l2ZfPktUNUXQ= +github.com/aws/aws-sdk-go-v2/service/kinesis v1.8.0 h1:Cz26j4wGD1tJ2w/M8iLhaS81AkAGY3gEYRt0xQWjEIs= +github.com/aws/aws-sdk-go-v2/service/kinesis v1.8.0/go.mod h1:QyNCg1xtWFJVL++i6ZyVcwXZCiKTNeXHH9zZu3NHOdU= +github.com/aws/aws-sdk-go-v2/service/sso v1.3.3 h1:K2gCnGvAASpz+jqP9iyr+F/KNjmTYf8aWOtTQzhmZ5w= +github.com/aws/aws-sdk-go-v2/service/sso v1.3.3/go.mod h1:Jgw5O+SK7MZ2Yi9Yvzb4PggAPYaFSliiQuWR0hNjexk= +github.com/aws/aws-sdk-go-v2/service/sso v1.5.0 h1:VnrCAJTp1bDxU79UuW/D4z7bwZ7xOc7JjDKpqXL/m04= +github.com/aws/aws-sdk-go-v2/service/sso v1.5.0/go.mod h1:GsqaJOJeOfeYD88/2vHWKXegvDRofDqWwC5i48A2kgs= +github.com/aws/aws-sdk-go-v2/service/sso v1.6.0 h1:JDgKIUZOmLFu/Rv6zXLrVTWCmzA0jcTdvsT8iFIKrAI= +github.com/aws/aws-sdk-go-v2/service/sso v1.6.0/go.mod h1:Q/l0ON1annSU+mc0JybDy1Gy6dnJxIcWjphO6qJPzvM= +github.com/aws/aws-sdk-go-v2/service/sts v1.6.2 h1:l504GWCoQi1Pk68vSUFGLmDIEMzRfVGNgLakDK+Uj58= +github.com/aws/aws-sdk-go-v2/service/sts v1.6.2/go.mod h1:RBhoMJB8yFToaCnbe0jNq5Dcdy0jp6LhHqg55rjClkM= +github.com/aws/aws-sdk-go-v2/service/sts v1.8.0 h1:7N7RsEVvUcvEg7jrWKU5AnSi4/6b6eY9+wG1g6W4ExE= +github.com/aws/aws-sdk-go-v2/service/sts v1.8.0/go.mod h1:dOlm91B439le5y1vtPCk5yJtbx3RdT3hRGYRY8TYKvQ= +github.com/aws/aws-sdk-go-v2/service/sts v1.9.0 h1:rBLCnL8hQ7Sv1S4XCPYgTMI7Uhg81BkvzIiK+/of2zY= +github.com/aws/aws-sdk-go-v2/service/sts v1.9.0/go.mod h1:jLKCFqS+1T4i7HDqCP9GM4Uk75YW1cS0o82LdxpMyOE= +github.com/aws/smithy-go v1.7.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= +github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= +github.com/aws/smithy-go v1.8.1 h1:9Y6qxtzgEODaLNGN+oN2QvcHvKUe4jsH8w4M+8LXzGk= +github.com/aws/smithy-go v1.8.1/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= +github.com/aws/smithy-go v1.9.0 h1:c7FUdEqrQA1/UVKKCNDFQPNKGp4FQg3YW4Ck5SLTG58= +github.com/aws/smithy-go v1.9.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/awslabs/kinesis-aggregation/go v0.0.0-20210630091500-54e17340d32f h1:Pf0BjJDga7C98f0vhw+Ip5EaiE07S3lTKpIYPNS0nMo= github.com/awslabs/kinesis-aggregation/go v0.0.0-20210630091500-54e17340d32f/go.mod h1:SghidfnxvX7ribW6nHI7T+IBbc9puZ9kk5Tx/88h8P4= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= @@ -51,38 +164,57 @@ github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/census-instrumentation/opencensus-proto v0.2.1 h1:glEXhBS5PSLLv4IXzLA5yPRVX4bilULVyxxbrfOtDAk= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chzyer/logex v1.1.10 h1:Swpa1K6QvQznwJRcfTfQJmTE72DqScAa40E+fbHEXEE= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e h1:fY5BOSpyZCqRo5OhCuC+XN+r/bBCmeuuJtjz+bCNIf8= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1 h1:q763qf9huN11kDQavWsoZXJNW3xEE4JJyHa5Q25/sd8= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f h1:WBZRG4aNOuI15bLRrCgN8fCq8E5Xuty6jGbmSNEvSsU= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4 h1:rEvIZUSZ3fx39WIi3JkQqQBitGwpELBIYWeBVh6wn+E= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0 h1:EQciDnbrYxy13PgWoY8AqoxGiPrpgBZ1R8UNe3ddc+A= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1 h1:QbL/5oDUmRBzO9/Z7Seo6zf912W/a6Sr4Eu0G/3Jho0= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4 h1:WtGNWLvXpe6ZudgnXrq0barxBImvnnJoMEhXAzcbM0I= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0 h1:wDJmvq38kDhkVxi50ni9ykkdUr1PKgqKOoi01fa0Mdk= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0 h1:DGJh0Sm43HbOeYDNnVZFl8BvcYVvjD5bqYJvp0REbwQ= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0 h1:TrB8swr/68K7m9CcGut2g3UOihhbcbiMAYiuTXdEih4= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/godbus/dbus/v5 v5.0.4 h1:9349emZab16e7zQvpmsbtjc18ykshndd8y2PG3sgJbA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= +github.com/gogo/protobuf v1.1.1 h1:72R+M5VuhED/KujmZVcIquuo8mBgX4oVda//DQb3PXo= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -90,6 +222,7 @@ github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFU github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -109,6 +242,7 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -120,8 +254,13 @@ github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6 h1:BKbKCqvP6I+rmFHt06ZmyQtvB8xAkWdhFyr0ZUNZcxQ= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/martian/v3 v3.0.0 h1:pMen7vLs8nvgEYhywH3KDWJIJTeEr2ULsVWHWYHQyBs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= @@ -129,44 +268,60 @@ github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hf github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99 h1:Ak8CrdlwwXwAZxzS66vgPt4U8yUZX7JwLvVR58FN5jM= github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1 h1:0hERBMJE1eitiLkihrMvRVBYAkpHzc/J3QdDN+dAcgU= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6 h1:UDMh68UUwekSh5iP2OMhRRZJiiBccgV7axzUG8vi56c= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11 h1:uVUAXhF2To8cbw/3xN3pxj6kk7TYKs98NIrTqPlMWAQ= github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1 h1:6QPYqodiu3GuPL+7mfx+NwDdp2eTkp9IfEUpgAwUN0o= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pty v1.1.1 h1:VkoXIwSboBpnk99O/KFauAEILuNHv5DVFKZMBN/gUgw= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -195,17 +350,24 @@ github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4O github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0VU= github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/rogpeppe/go-internal v1.3.0 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rs/xid v1.3.0 h1:6NjYksEUlhurdVehpc7S7dk6DAmcKv8V9gG0FsVN2U4= github.com/rs/xid v1.3.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/rs/zerolog v1.25.0 h1:Rj7XygbUHKUlDPcVdoLyR91fJBsduXj5fRxyqIQj/II= github.com/rs/zerolog v1.25.0/go.mod h1:7KHcEGe0QZPOm2IE4Kpb5rTh6n1h2hIgS5OOnu1rUaI= +github.com/rs/zerolog v1.26.0 h1:ORM4ibhEZeTeQlCojCK2kPz1ogAY4bGs4tD+SaAdGaE= +github.com/rs/zerolog v1.26.0/go.mod h1:yBiM87lvSqX8h0Ww4sdzNSkVYZ8dL2xjZJG1lAuGZEo= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.3.0 h1:NGXK3lHquSN08v5vWalVI/L8XU9hdzE/G6xsrze47As= +github.com/stretchr/objx v0.3.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= @@ -214,11 +376,15 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.3.5 h1:dPmz1Snjq0kmkz159iL7S6WzdahUTHnHB5M56WFVifs= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.0 h1:OtISOGfH6sOWa1/qXqqAiOIAO6Z5J3AEAE18WAq6BiQ= +github.com/yuin/goldmark v1.4.0/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.4 h1:LYy1Hy3MJdrCdMwwzxA/dRok4ejH+RwNGbuoD9fCjto= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= @@ -235,6 +401,7 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -245,8 +412,10 @@ golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6 h1:QE6XYQK6naiK1EPAe1g/ILLxN5RBoH5xkJk3CqlMI/Y= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b h1:+qEpEAPhDZ1o0x3tHzZTQDArnOixOzGD9HUJfcg0mb4= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -257,8 +426,10 @@ golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028 h1:4+4C/Iv2U4fMZBiMCc98MG1In4gJY5YRhtpDNeDeHWs= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= @@ -266,6 +437,7 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2 h1:Gz96sIWK3OalVv/I/qNygP42zyoKp3xptRVCWRFEBvo= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -296,14 +468,18 @@ golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210525063256-abc453219eb5 h1:wjuX4b5yYQnEQHzd+CBcrcC6OVR2J1CN6mUy0oSxIPo= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210614182718-04defd469f4e h1:XpT3nA5TvE525Ne3hInMh6+GETgn27Zfm9dxsThnX2Q= golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d h1:20cMwl2fHAzkJMEA+8J4JgqBQcQGzbisXo31MIeenXI= +golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c h1:pkQiBZBvdos9qq4wBAHqlzuZHEXo07pqV06ef90u1WI= golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -314,6 +490,7 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -353,8 +530,20 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac h1:oN6lz7iLW/YC7un8pq+9bOLyXrprv2+DKfkJY+2LJJw= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359 h1:2B5p2L5IfGiD7+b9BOoRMC6DgObAVZV+Fsp050NqXik= +golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211029165221-6e7872819dc8 h1:M69LAlWZCshgp0QSzyDcSsSIejIEeuaCVpmwcKwyLMk= +golang.org/x/sys v0.0.0-20211029165221-6e7872819dc8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211102061401-a2f17f7b995c h1:QOfDMdrf/UwlVR0UBq2Mpr58UzNtvgJRXA4BgPfFACs= +golang.org/x/sys v0.0.0-20211102061401-a2f17f7b995c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211105183446-c75c47738b0c h1:+8miTPjMCTXwih7BQmvWwd0PjdBZq2MKp/qQaahSzEM= +golang.org/x/sys v0.0.0-20211105183446-c75c47738b0c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211106132015-ebca88c72f68 h1:Ywe/f3fNleF8I6F6qv3MeFoSZ6CTf2zBMMa/7qVML8M= +golang.org/x/sys v0.0.0-20211106132015-ebca88c72f68/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1 h1:v+OssWQX+hTHEmOBgwxdZxK4zHq3yOs8F9J7mk0PY8E= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -365,6 +554,7 @@ golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -406,7 +596,10 @@ golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.1.5 h1:ouewzE6p+/VEB31YYnTbEJdi8pFqKp4P4n85vwo3DHA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.7 h1:6j8CgantCy3yc8JGBqkDLMKWqZ0RDU2g1HVgacojGWQ= +golang.org/x/tools v0.1.7/go.mod h1:LGqMHiF4EqQNHR1JncWGqT5BVaXmza+X+BDGol+dOxo= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -427,12 +620,14 @@ google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/ google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= +google.golang.org/api v0.30.0 h1:yfrXXP61wVuLb0vBcG6qaOoIoqYEzOQS8jum51jkv2w= google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.6 h1:lMO5rYAqUxkmaj76jAkRUvt5JZgFymx/+Q5Mzfivuhc= google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -462,6 +657,7 @@ google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEY google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200825200019-8632dd797987 h1:PDIOdWxZ8eRizhKa1AAvY53xsvLB1cWorMjslvY3VA8= google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= @@ -474,6 +670,7 @@ google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.0 h1:T7P4R73V3SSDPhH7WW7ATbfViLtmamH0DKrP3f9AuDI= google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -489,11 +686,13 @@ google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp0 google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0 h1:0vLT13EuvQ0hNvakwLuFZ/jYrLp5F3kcWHXdRggjCE8= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= @@ -514,7 +713,11 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4 h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0 h1:HfqmD5MEmC0zvwBuF187nq9mdnXjXsSivRiXN7SmRkE= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/quote/v3 v3.1.0 h1:9JKUTTIUgS6kzR9mK1YuGKv6Nl+DijDNIc0ghT58FaY= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0 h1:7uVkIFmeBqHfdjD+gZwtXXI+RODJ2Wc4O7MPEh/QiW4= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/test/lease_stealing_util_test.go b/test/lease_stealing_util_test.go index 21b8ab3..aab57ae 100644 --- a/test/lease_stealing_util_test.go +++ b/test/lease_stealing_util_test.go @@ -1,16 +1,18 @@ package test import ( + "context" "fmt" "sync" "testing" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/dynamodb" - "github.com/aws/aws-sdk-go/service/dynamodb/dynamodbiface" - "github.com/aws/aws-sdk-go/service/kinesis/kinesisiface" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/dynamodb" + "github.com/aws/aws-sdk-go-v2/service/dynamodb/types" + "github.com/aws/aws-sdk-go-v2/service/kinesis" "github.com/stretchr/testify/assert" + chk "github.com/vmware/vmware-go-kcl/clientlibrary/checkpoint" cfg "github.com/vmware/vmware-go-kcl/clientlibrary/config" wk "github.com/vmware/vmware-go-kcl/clientlibrary/worker" @@ -18,10 +20,10 @@ import ( type LeaseStealingTest struct { t *testing.T - config *TestClusterConfig + config *TestClusterConfig cluster *TestCluster - kc kinesisiface.KinesisAPI - dc dynamodbiface.DynamoDBAPI + kc *kinesis.Client + dc *dynamodb.Client backOffSeconds int maxRetries int @@ -83,21 +85,19 @@ func (lst *LeaseStealingTest) getShardCountByWorker() map[string]int { } shardsByWorker := map[string]map[string]bool{} - err := lst.dc.ScanPages(input, func(out *dynamodb.ScanOutput, lastPage bool) bool { - for _, result := range out.Items { - if shardID, ok := result[chk.LeaseKeyKey]; !ok { - continue - } else if assignedTo, ok := result[chk.LeaseOwnerKey]; !ok { - continue - } else { - if _, ok := shardsByWorker[*assignedTo.S]; !ok { - shardsByWorker[*assignedTo.S] = map[string]bool{} - } - shardsByWorker[*assignedTo.S][*shardID.S] = true + scan, err := lst.dc.Scan(context.TODO(), input) + for _, result := range scan.Items { + if shardID, ok := result[chk.LeaseKeyKey]; !ok { + continue + } else if assignedTo, ok := result[chk.LeaseOwnerKey]; !ok { + continue + } else { + if _, ok := shardsByWorker[assignedTo.(*types.AttributeValueMemberS).Value]; !ok { + shardsByWorker[assignedTo.(*types.AttributeValueMemberS).Value] = map[string]bool{} } + shardsByWorker[assignedTo.(*types.AttributeValueMemberS).Value][shardID.(*types.AttributeValueMemberS).Value] = true } - return !lastPage - }) + } assert.Nil(lst.t, err) shardCountByWorker := map[string]int{} @@ -108,12 +108,12 @@ func (lst *LeaseStealingTest) getShardCountByWorker() map[string]int { } type LeaseStealingAssertions struct { - expectedLeasesForIntialWorker int - expectedLeasesPerWorker int + expectedLeasesForInitialWorker int + expectedLeasesPerWorker int } func (lst *LeaseStealingTest) Run(assertions LeaseStealingAssertions) { - // Publish records onto stream thoughtout the entire duration of the test + // Publish records onto stream throughout the entire duration of the test stop := lst.publishSomeData() defer stop() @@ -126,16 +126,16 @@ func (lst *LeaseStealingTest) Run(assertions LeaseStealingAssertions) { time.Sleep(time.Duration(lst.backOffSeconds) * time.Second) shardCountByWorker := lst.getShardCountByWorker() - if shardCount, ok := shardCountByWorker[worker1]; ok && shardCount == assertions.expectedLeasesForIntialWorker { + if shardCount, ok := shardCountByWorker[worker1]; ok && shardCount == assertions.expectedLeasesForInitialWorker { worker1ShardCount = shardCount break } } // Assert correct number of leases - assert.Equal(lst.t, assertions.expectedLeasesForIntialWorker, worker1ShardCount) + assert.Equal(lst.t, assertions.expectedLeasesForInitialWorker, worker1ShardCount) - // Spawn Remaining Wokers + // Spawn Remaining Workers for i := 0; i < lst.config.numWorkers-1; i++ { lst.cluster.SpawnWorker() } diff --git a/test/record_processor_test.go b/test/record_processor_test.go index 2e37368..ff4fef3 100644 --- a/test/record_processor_test.go +++ b/test/record_processor_test.go @@ -23,7 +23,9 @@ package test import ( "testing" + "github.com/aws/aws-sdk-go-v2/aws" "github.com/stretchr/testify/assert" + kc "github.com/vmware/vmware-go-kcl/clientlibrary/interfaces" ) @@ -50,7 +52,7 @@ type dumpRecordProcessor struct { } func (dd *dumpRecordProcessor) Initialize(input *kc.InitializationInput) { - dd.t.Logf("Processing SharId: %v at checkpoint: %v", input.ShardId, aws.StringValue(input.ExtendedSequenceNumber.SequenceNumber)) + dd.t.Logf("Processing SharId: %v at checkpoint: %v", input.ShardId, aws.ToString(input.ExtendedSequenceNumber.SequenceNumber)) shardID = input.ShardId dd.count = 0 } @@ -76,18 +78,18 @@ func (dd *dumpRecordProcessor) ProcessRecords(input *kc.ProcessRecordsInput) { // Calculate the time taken from polling records and delivering to record processor for a batch. diff := input.CacheExitTime.Sub(*input.CacheEntryTime) dd.t.Logf("Checkpoint progress at: %v, MillisBehindLatest = %v, KCLProcessTime = %v", lastRecordSequenceNumber, input.MillisBehindLatest, diff) - input.Checkpointer.Checkpoint(lastRecordSequenceNumber) + _ = input.Checkpointer.Checkpoint(lastRecordSequenceNumber) } func (dd *dumpRecordProcessor) Shutdown(input *kc.ShutdownInput) { - dd.t.Logf("Shutdown Reason: %v", aws.StringValue(kc.ShutdownReasonMessage(input.ShutdownReason))) + dd.t.Logf("Shutdown Reason: %v", aws.ToString(kc.ShutdownReasonMessage(input.ShutdownReason))) dd.t.Logf("Processed Record Count = %d", dd.count) // When the value of {@link ShutdownInput#getShutdownReason()} is // {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason#TERMINATE} it is required that you // checkpoint. Failure to do so will result in an IllegalArgumentException, and the KCL no longer making progress. if input.ShutdownReason == kc.TERMINATE { - input.Checkpointer.Checkpoint(nil) + _ = input.Checkpointer.Checkpoint(nil) } assert.True(dd.t, dd.count > 0) diff --git a/test/record_publisher_test.go b/test/record_publisher_test.go index baaac57..85017a3 100644 --- a/test/record_publisher_test.go +++ b/test/record_publisher_test.go @@ -19,61 +19,99 @@ package test import ( + "context" "crypto/md5" "fmt" "sync" + "testing" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/credentials" - "github.com/aws/aws-sdk-go/aws/session" - "github.com/aws/aws-sdk-go/service/dynamodb" - "github.com/aws/aws-sdk-go/service/kinesis" - "github.com/aws/aws-sdk-go/service/kinesis/kinesisiface" - rec "github.com/awslabs/kinesis-aggregation/go/records" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/aws/retry" + awsConfig "github.com/aws/aws-sdk-go-v2/config" + "github.com/aws/aws-sdk-go-v2/credentials" + "github.com/aws/aws-sdk-go-v2/service/dynamodb" + "github.com/aws/aws-sdk-go-v2/service/kinesis" + "github.com/aws/aws-sdk-go-v2/service/kinesis/types" "github.com/golang/protobuf/proto" - "github.com/vmware/vmware-go-kcl/clientlibrary/utils" - "testing" + "github.com/vmware/vmware-go-kcl/clientlibrary/utils" + rec "github.com/vmware/vmware-go-kcl/internal/records" ) const specstr = `{"name":"kube-qQyhk","networking":{"containerNetworkCidr":"10.2.0.0/16"},"orgName":"BVT-Org-cLQch","projectName":"project-tDSJd","serviceLevel":"DEVELOPER","size":{"count":1},"version":"1.8.1-4"}` // NewKinesisClient to create a Kinesis Client. -func NewKinesisClient(t *testing.T, regionName, endpoint string, credentials *credentials.Credentials) *kinesis.Kinesis { - s, err := session.NewSession(&aws.Config{ - Region: aws.String(regionName), - Endpoint: aws.String(endpoint), - Credentials: credentials, +func NewKinesisClient(t *testing.T, regionName, endpoint string, creds *credentials.StaticCredentialsProvider) *kinesis.Client { + // create session for Kinesis + t.Logf("Creating Kinesis client") + + resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) { + return aws.Endpoint { + PartitionID: "aws", + URL: endpoint, + SigningRegion: regionName, + }, nil }) + cfg, err := awsConfig.LoadDefaultConfig( + context.TODO(), + awsConfig.WithRegion(regionName), + awsConfig.WithCredentialsProvider( + credentials.NewStaticCredentialsProvider( + creds.Value.AccessKeyID, + creds.Value.SecretAccessKey, + creds.Value.SessionToken)), + awsConfig.WithEndpointResolver(resolver), + awsConfig.WithRetryer(func() aws.Retryer { + return retry.AddWithMaxBackoffDelay(retry.NewStandard(), retry.DefaultMaxBackoff) + }), + ) + if err != nil { // no need to move forward - t.Fatalf("Failed in getting Kinesis session for creating Worker: %+v", err) + t.Fatalf("Failed in loading Kinesis default config for creating Worker: %+v", err) } - return kinesis.New(s) + + return kinesis.NewFromConfig(cfg) } // NewDynamoDBClient to create a Kinesis Client. -func NewDynamoDBClient(t *testing.T, regionName, endpoint string, credentials *credentials.Credentials) *dynamodb.DynamoDB { - s, err := session.NewSession(&aws.Config{ - Region: aws.String(regionName), - Endpoint: aws.String(endpoint), - Credentials: credentials, +func NewDynamoDBClient(t *testing.T, regionName, endpoint string, creds *credentials.StaticCredentialsProvider) *dynamodb.Client { + resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) { + return aws.Endpoint { + PartitionID: "aws", + URL: endpoint, + SigningRegion: regionName, + }, nil }) + cfg, err := awsConfig.LoadDefaultConfig( + context.TODO(), + awsConfig.WithRegion(regionName), + awsConfig.WithCredentialsProvider( + credentials.NewStaticCredentialsProvider( + creds.Value.AccessKeyID, + creds.Value.SecretAccessKey, + creds.Value.SessionToken)), + awsConfig.WithEndpointResolver(resolver), + awsConfig.WithRetryer(func() aws.Retryer { + return retry.AddWithMaxBackoffDelay(retry.NewStandard(), retry.DefaultMaxBackoff) + }), + ) + if err != nil { - // no need to move forward - t.Fatalf("Failed in getting DynamoDB session for creating Worker: %+v", err) + t.Fatalf("unable to load SDK config, %v", err) } - return dynamodb.New(s) + + return dynamodb.NewFromConfig(cfg) } -func continuouslyPublishSomeData(t *testing.T, kc kinesisiface.KinesisAPI) func() { - shards := []*kinesis.Shard{} +func continuouslyPublishSomeData(t *testing.T, kc *kinesis.Client) func() { + var shards []types.Shard var nextToken *string for { - out, err := kc.ListShards(&kinesis.ListShardsInput{ + out, err := kc.ListShards(context.TODO(), &kinesis.ListShardsInput { StreamName: aws.String(streamName), NextToken: nextToken, }) @@ -112,7 +150,7 @@ func continuouslyPublishSomeData(t *testing.T, kc kinesisiface.KinesisAPI) func( } } -func publishToAllShards(t *testing.T, kc kinesisiface.KinesisAPI, shards []*kinesis.Shard) { +func publishToAllShards(t *testing.T, kc *kinesis.Client, shards []types.Shard) { // Put records to all shards for i := 0; i < 10; i++ { for _, shard := range shards { @@ -122,7 +160,7 @@ func publishToAllShards(t *testing.T, kc kinesisiface.KinesisAPI, shards []*kine } // publishSomeData to put some records into Kinesis stream -func publishSomeData(t *testing.T, kc kinesisiface.KinesisAPI) { +func publishSomeData(t *testing.T, kc *kinesis.Client) { // Put some data into stream. t.Log("Putting data into stream using PutRecord API...") for i := 0; i < 50; i++ { @@ -146,8 +184,8 @@ func publishSomeData(t *testing.T, kc kinesisiface.KinesisAPI) { } // publishRecord to put a record into Kinesis stream using PutRecord API. -func publishRecord(t *testing.T, kc kinesisiface.KinesisAPI, hashKey *string) { - input := &kinesis.PutRecordInput{ +func publishRecord(t *testing.T, kc *kinesis.Client, hashKey *string) { + input := &kinesis.PutRecordInput { Data: []byte(specstr), StreamName: aws.String(streamName), PartitionKey: aws.String(utils.RandStringBytesMaskImpr(10)), @@ -156,7 +194,7 @@ func publishRecord(t *testing.T, kc kinesisiface.KinesisAPI, hashKey *string) { input.ExplicitHashKey = hashKey } // Use random string as partition key to ensure even distribution across shards - _, err := kc.PutRecord(input) + _, err := kc.PutRecord(context.TODO(), input) if err != nil { t.Errorf("Error in PutRecord. %+v", err) @@ -164,19 +202,19 @@ func publishRecord(t *testing.T, kc kinesisiface.KinesisAPI, hashKey *string) { } // publishRecord to put a record into Kinesis stream using PutRecords API. -func publishRecords(t *testing.T, kc kinesisiface.KinesisAPI) { +func publishRecords(t *testing.T, kc *kinesis.Client) { // Use random string as partition key to ensure even distribution across shards - records := make([]*kinesis.PutRecordsRequestEntry, 5) + records := make([]types.PutRecordsRequestEntry, 5) for i := 0; i < 5; i++ { - record := &kinesis.PutRecordsRequestEntry{ + record := types.PutRecordsRequestEntry { Data: []byte(specstr), PartitionKey: aws.String(utils.RandStringBytesMaskImpr(10)), } records[i] = record } - _, err := kc.PutRecords(&kinesis.PutRecordsInput{ + _, err := kc.PutRecords(context.TODO(), &kinesis.PutRecordsInput{ Records: records, StreamName: aws.String(streamName), }) @@ -187,10 +225,10 @@ func publishRecords(t *testing.T, kc kinesisiface.KinesisAPI) { } // publishRecord to put a record into Kinesis stream using PutRecord API. -func publishAggregateRecord(t *testing.T, kc kinesisiface.KinesisAPI) { +func publishAggregateRecord(t *testing.T, kc *kinesis.Client) { data := generateAggregateRecord(5, specstr) // Use random string as partition key to ensure even distribution across shards - _, err := kc.PutRecord(&kinesis.PutRecordInput{ + _, err := kc.PutRecord(context.TODO(), &kinesis.PutRecordInput { Data: data, StreamName: aws.String(streamName), PartitionKey: aws.String(utils.RandStringBytesMaskImpr(10)), diff --git a/test/worker_custom_test.go b/test/worker_custom_test.go index 19a6fb7..a513b56 100644 --- a/test/worker_custom_test.go +++ b/test/worker_custom_test.go @@ -19,14 +19,14 @@ package test import ( + "context" "os" "sync" "testing" "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/session" - "github.com/aws/aws-sdk-go/service/kinesis" + "github.com/aws/aws-sdk-go-v2/config" + "github.com/aws/aws-sdk-go-v2/service/kinesis" log "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" @@ -77,7 +77,8 @@ func TestWorkerInjectCheckpointer(t *testing.T) { ID: shardID, Mux: &sync.RWMutex{}, } - checkpointer.FetchCheckpoint(status) + + _ = checkpointer.FetchCheckpoint(status) // checkpointer should be the same assert.NotEmpty(t, status.Checkpoint) @@ -104,12 +105,13 @@ func TestWorkerInjectKinesis(t *testing.T) { // configure cloudwatch as metrics system kclConfig.WithMonitoringService(getMetricsConfig(kclConfig, metricsSystem)) - // create custom Kinesis - s, err := session.NewSession(&aws.Config{ - Region: aws.String(regionName), - }) + defaultConfig, err := config.LoadDefaultConfig( + context.TODO(), + config.WithRegion(regionName), + ) + assert.Nil(t, err) - kc := kinesis.New(s) + kc := kinesis.NewFromConfig(defaultConfig) // Put some data into stream. // publishSomeData(t, kc) @@ -146,11 +148,13 @@ func TestWorkerInjectKinesisAndCheckpointer(t *testing.T) { kclConfig.WithMonitoringService(getMetricsConfig(kclConfig, metricsSystem)) // create custom Kinesis - s, err := session.NewSession(&aws.Config{ - Region: aws.String(regionName), - }) + defaultConfig, err := config.LoadDefaultConfig( + context.TODO(), + config.WithRegion(regionName), + ) + assert.Nil(t, err) - kc := kinesis.New(s) + kc := kinesis.NewFromConfig(defaultConfig) // Put some data into stream. // publishSomeData(t, kc) diff --git a/test/worker_lease_stealing_test.go b/test/worker_lease_stealing_test.go index c35974c..3742a8f 100644 --- a/test/worker_lease_stealing_test.go +++ b/test/worker_lease_stealing_test.go @@ -20,8 +20,8 @@ func TestLeaseStealing(t *testing.T) { } test := NewLeaseStealingTest(t, config, newLeaseStealingWorkerFactory(t)) test.Run(LeaseStealingAssertions{ - expectedLeasesForIntialWorker: config.numShards, - expectedLeasesPerWorker: config.numShards / config.numWorkers, + expectedLeasesForInitialWorker: config.numShards, + expectedLeasesPerWorker: config.numShards / config.numWorkers, }) } @@ -55,7 +55,7 @@ func (wf *leaseStealingWorkerFactory) CreateKCLConfig(workerID string, config *T WithLogger(log) } -func (wf *leaseStealingWorkerFactory) CreateWorker(workerID string, kclConfig *cfg.KinesisClientLibConfiguration) *wk.Worker { +func (wf *leaseStealingWorkerFactory) CreateWorker(_ string, kclConfig *cfg.KinesisClientLibConfiguration) *wk.Worker { worker := wk.NewWorker(recordProcessorFactory(wf.t), kclConfig) return worker } @@ -71,8 +71,8 @@ func TestLeaseStealingInjectCheckpointer(t *testing.T) { } test := NewLeaseStealingTest(t, config, newleaseStealingWorkerFactoryCustomChk(t)) test.Run(LeaseStealingAssertions{ - expectedLeasesForIntialWorker: config.numShards, - expectedLeasesPerWorker: config.numShards / config.numWorkers, + expectedLeasesForInitialWorker: config.numShards, + expectedLeasesPerWorker: config.numShards / config.numWorkers, }) } @@ -101,10 +101,10 @@ func TestLeaseStealingWithMaxLeasesForWorker(t *testing.T) { regionName: regionName, workerIDTemplate: workerID + "-%v", } - test := NewLeaseStealingTest(t, config, newleaseStealingWorkerFactoryMaxLeases(t, config.numShards-1)) + test := NewLeaseStealingTest(t, config, newLeaseStealingWorkerFactoryMaxLeases(t, config.numShards-1)) test.Run(LeaseStealingAssertions{ - expectedLeasesForIntialWorker: config.numShards - 1, - expectedLeasesPerWorker: 2, + expectedLeasesForInitialWorker: config.numShards - 1, + expectedLeasesPerWorker: 2, }) } @@ -113,7 +113,7 @@ type leaseStealingWorkerFactoryMaxLeases struct { *leaseStealingWorkerFactory } -func newleaseStealingWorkerFactoryMaxLeases(t *testing.T, maxLeases int) *leaseStealingWorkerFactoryMaxLeases { +func newLeaseStealingWorkerFactoryMaxLeases(t *testing.T, maxLeases int) *leaseStealingWorkerFactoryMaxLeases { return &leaseStealingWorkerFactoryMaxLeases{ maxLeases, newLeaseStealingWorkerFactory(t), diff --git a/test/worker_test.go b/test/worker_test.go index a445a59..680a7c6 100644 --- a/test/worker_test.go +++ b/test/worker_test.go @@ -26,9 +26,7 @@ import ( "testing" "time" - "github.com/aws/aws-sdk-go/aws/credentials" - "github.com/aws/aws-sdk-go/aws/credentials/stscreds" - "github.com/aws/aws-sdk-go/aws/session" + "github.com/aws/aws-sdk-go-v2/credentials" "github.com/prometheus/common/expfmt" "github.com/stretchr/testify/assert" @@ -47,6 +45,8 @@ const ( regionName = "us-west-2" workerID = "test-worker" consumerName = "enhanced-fan-out-consumer" + kinesisEndpoint = "https://kinesis.eu-west-1.amazonaws.com" + dynamoEndpoint = "https://dynamodb.eu-west-1.amazonaws.com" ) const metricsSystem = "cloudwatch" @@ -76,7 +76,8 @@ func TestWorker(t *testing.T) { WithMaxLeasesForWorker(1). WithShardSyncIntervalMillis(5000). WithFailoverTimeMillis(300000). - WithLogger(log) + WithLogger(log). + WithKinesisEndpoint(kinesisEndpoint) runTest(kclConfig, false, t) } @@ -98,7 +99,8 @@ func TestWorkerWithTimestamp(t *testing.T) { WithMaxLeasesForWorker(1). WithShardSyncIntervalMillis(5000). WithFailoverTimeMillis(300000). - WithLogger(log) + WithLogger(log). + WithKinesisEndpoint(kinesisEndpoint) runTest(kclConfig, false, t) } @@ -128,24 +130,28 @@ func TestWorkerWithSigInt(t *testing.T) { WithMaxLeasesForWorker(1). WithShardSyncIntervalMillis(5000). WithFailoverTimeMillis(300000). - WithLogger(log) + WithLogger(log). + WithKinesisEndpoint(kinesisEndpoint) runTest(kclConfig, true, t) } func TestWorkerStatic(t *testing.T) { - t.Skip("Need to provide actual credentials") + //t.Skip("Need to provide actual credentials") // Fill in the credentials for accessing Kinesis and DynamoDB. // Note: use empty string as SessionToken for long-term credentials. - creds := credentials.NewStaticCredentials("AccessKeyId", "SecretAccessKey", "SessionToken") + kinesisCreds := credentials.NewStaticCredentialsProvider("", "", "") + dynamoCreds := credentials.NewStaticCredentialsProvider("", "", "") - kclConfig := cfg.NewKinesisClientLibConfigWithCredential(appName, streamName, regionName, workerID, creds). + kclConfig := cfg.NewKinesisClientLibConfigWithCredentials(appName, streamName, regionName, workerID, &kinesisCreds, &dynamoCreds). WithInitialPositionInStream(cfg.LATEST). WithMaxRecords(10). WithMaxLeasesForWorker(1). WithShardSyncIntervalMillis(5000). - WithFailoverTimeMillis(300000) + WithFailoverTimeMillis(300000). + WithKinesisEndpoint(kinesisEndpoint). + WithDynamoDBEndpoint(dynamoEndpoint) runTest(kclConfig, false, t) } @@ -155,25 +161,29 @@ func TestWorkerAssumeRole(t *testing.T) { // Initial credentials loaded from SDK's default credential chain. Such as // the environment, shared credentials (~/.aws/credentials), or EC2 Instance - // Role. These credentials will be used to to make the STS Assume Role API. - sess := session.Must(session.NewSession()) + // Role. These credentials will be used to make the STS Assume Role API. + //sess := session.Must(session.NewSession()) // Create the credentials from AssumeRoleProvider to assume the role // referenced by the "myRoleARN" ARN. - creds := stscreds.NewCredentials(sess, "arn:aws:iam::*:role/kcl-test-publisher") + //kinesisCreds := stscreds.NewAssumeRoleProvider(sess, "arn:aws:iam::*:role/kcl-test-publisher") + kinesisCreds := credentials.NewStaticCredentialsProvider("", "", "") + dynamoCreds := credentials.NewStaticCredentialsProvider("", "", "") - kclConfig := cfg.NewKinesisClientLibConfigWithCredential(appName, streamName, regionName, workerID, creds). + kclConfig := cfg.NewKinesisClientLibConfigWithCredentials(appName, streamName, regionName, workerID, &kinesisCreds, &dynamoCreds). WithInitialPositionInStream(cfg.LATEST). WithMaxRecords(10). WithMaxLeasesForWorker(1). WithShardSyncIntervalMillis(5000). - WithFailoverTimeMillis(300000) + WithFailoverTimeMillis(300000). + WithKinesisEndpoint(kinesisEndpoint). + WithDynamoDBEndpoint(dynamoEndpoint) runTest(kclConfig, false, t) } func TestEnhancedFanOutConsumer(t *testing.T) { - // At miminal, use standard logrus logger + // At minimal, use standard logrus logger // log := logger.NewLogrusLogger(logrus.StandardLogger()) // // In order to have precise control over logging. Use logger with config @@ -202,7 +212,7 @@ func TestEnhancedFanOutConsumer(t *testing.T) { } func TestEnhancedFanOutConsumerDefaultConsumerName(t *testing.T) { - // At miminal, use standard logrus logger + // At minimal, use standard logrus logger // log := logger.NewLogrusLogger(logrus.StandardLogger()) // // In order to have precise control over logging. Use logger with config @@ -234,7 +244,7 @@ func TestEnhancedFanOutConsumerARN(t *testing.T) { t.Skip("Need to provide actual consumerARN") consumerARN := "arn:aws:kinesis:*:stream/kcl-test/consumer/fanout-poc-consumer-test:*" - // At miminal, use standard logrus logger + // At minimal, use standard logrus logger // log := logger.NewLogrusLogger(logrus.StandardLogger()) // // In order to have precise control over logging. Use logger with config @@ -294,13 +304,14 @@ func runTest(kclConfig *cfg.KinesisClientLibConfiguration, triggersig bool, t *t if triggersig { t.Log("Trigger signal SIGINT") p, _ := os.FindProcess(os.Getpid()) - p.Signal(os.Interrupt) + _ = p.Signal(os.Interrupt) } // wait a few seconds before shutdown processing time.Sleep(30 * time.Second) - if metricsSystem == "prometheus" { + switch metricsSystem { + case "prometheus": res, err := http.Get("http://localhost:8080/metrics") if err != nil { t.Fatalf("Error scraping Prometheus endpoint %s", err) @@ -308,12 +319,12 @@ func runTest(kclConfig *cfg.KinesisClientLibConfiguration, triggersig bool, t *t var parser expfmt.TextParser parsed, err := parser.TextToMetricFamilies(res.Body) - res.Body.Close() + _ = res.Body.Close() if err != nil { t.Errorf("Error reading monitoring response %s", err) } - t.Logf("Prometheus: %+v", parsed) + t.Logf("Prometheus: %+v", parsed) } t.Log("Calling normal shutdown at the end of application.") @@ -327,7 +338,7 @@ func getMetricsConfig(kclConfig *cfg.KinesisClientLibConfiguration, service stri return cloudwatch.NewMonitoringServiceWithOptions(kclConfig.RegionName, kclConfig.KinesisCredentials, kclConfig.Logger, - cloudwatch.DEFAULT_CLOUDWATCH_METRICS_BUFFER_DURATION) + cloudwatch.DefaultCloudwatchMetricsBufferDuration) } if service == "prometheus" {