Skip to content
Merged
Show file tree
Hide file tree
Changes from 12 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
126 changes: 126 additions & 0 deletions pkg/app/api/grpcapi/piped_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"errors"
"time"

"github.com/google/uuid"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
Expand All @@ -42,6 +43,7 @@ import (
type PipedAPI struct {
applicationStore datastore.ApplicationStore
deploymentStore datastore.DeploymentStore
deploymentChainStore datastore.DeploymentChainStore
environmentStore datastore.EnvironmentStore
pipedStore datastore.PipedStore
projectStore datastore.ProjectStore
Expand All @@ -66,6 +68,7 @@ func NewPipedAPI(ctx context.Context, ds datastore.DataStore, sls stagelogstore.
a := &PipedAPI{
applicationStore: datastore.NewApplicationStore(ds),
deploymentStore: datastore.NewDeploymentStore(ds),
deploymentChainStore: datastore.NewDeploymentChainStore(ds),
environmentStore: datastore.NewEnvironmentStore(ds),
pipedStore: datastore.NewPipedStore(ds),
projectStore: datastore.NewProjectStore(ds),
Expand Down Expand Up @@ -963,6 +966,129 @@ func (a *PipedAPI) ReportUnregisteredApplicationConfigurations(ctx context.Conte
return nil, status.Errorf(codes.Unimplemented, "ReportUnregisteredApplicationConfigurations is not implemented yet")
}

// CreateDeploymentChain creates a new deployment chain object and all required commands to
// trigger deployment for applications in the chain.
func (a *PipedAPI) CreateDeploymentChain(ctx context.Context, req *pipedservice.CreateDeploymentChainRequest) (*pipedservice.CreateDeploymentChainResponse, error) {
projectID, _, _, err := rpcauth.ExtractPipedToken(ctx)
if err != nil {
return nil, err
}
Copy link
Member

@nghialv nghialv Nov 22, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a check to ensure that the application of the requested deployment belongs to that this authenticated Piped.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

addressed by 91a442b


buildChainNodes := func(matcher *pipedservice.CreateDeploymentChainRequest_ApplicationMatcher) ([]*model.ChainNode, []*model.Application, error) {
filters := []datastore.ListFilter{
{
Field: "ProjectId",
Operator: datastore.OperatorEqual,
Value: projectID,
},
}

if matcher.Name != "" {
filters = append(filters, datastore.ListFilter{
Field: "Name",
Operator: datastore.OperatorEqual,
Value: matcher.Name,
})
}

// TODO: Support find node apps by appKind and appLabels.

apps, _, err := a.applicationStore.ListApplications(ctx, datastore.ListOptions{
Filters: filters,
})
if err != nil {
return nil, nil, err
}

nodes := make([]*model.ChainNode, 0, len(apps))
for _, app := range apps {
nodes = append(nodes, &model.ChainNode{
ApplicationRef: &model.ChainApplicationRef{
ApplicationId: app.Id,
ApplicationName: app.Name,
},
})
}

return nodes, apps, nil
}

chainBlocks := make([]*model.ChainBlock, 0, len(req.Matchers)+1)
// Add the first deployment which created by piped as the first block of the chain.
chainBlocks = append(chainBlocks, &model.ChainBlock{
Index: 0,
Nodes: []*model.ChainNode{
{
ApplicationRef: &model.ChainApplicationRef{
ApplicationId: req.FirstDeployment.ApplicationId,
ApplicationName: req.FirstDeployment.ApplicationName,
},
DeploymentRef: &model.ChainDeploymentRef{
DeploymentId: req.FirstDeployment.Id,
},
},
},
})

apps := make([]*model.Application, 0)
for i, filter := range req.Matchers {
nodes, blockApps, err := buildChainNodes(filter)
if err != nil {
return nil, err
}

apps = append(apps, blockApps...)
chainBlocks = append(chainBlocks, &model.ChainBlock{
Index: int32(i + 1),
Nodes: nodes,
})
}

dc := model.DeploymentChain{
Id: uuid.New().String(),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we should add this chain ID to the deployment model.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You're right, we need that as a reference to display which chain this deployment belongs to in the deployment detailed page 👍

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

addressed by 050e72b

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The first deployment needs this ID as well. Do you think that we should create it by the first Piped?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, in such case, the first piped will create deploymentChain object and register it to the control-plane just as we do in trigger.triggerDeployment currently, right?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How do you think about adding a chain option to CreateDeploymentRequest and initializing chain commands in that API instead of creating this new CreateDeploymentChain?

In that way, we can have both chain ID and the first deployment ID at the same time.

Copy link
Member Author

@khanhtc1202 khanhtc1202 Nov 22, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So you mean to make CreateDeploymentRequest has the ability to make chain commands to trigger all other applications in the chain and the RPC call to create deployment for the first application will do it? In that case, I wonder 2 things:

  • we also need to provide matchers to filter out applications for chain block in that request body, right?
  • though it's named CreateDeployment, it may also create Command in some conditions.

Besides, not sure what do you mean by "we can have both chain ID and the first deployment ID at the same time", but since we separate the RPC to create the first deployment of the chain with the deployment chain itself, we have to first: create the deployment chain and wait until it's succeeded, then: use the deployment chain id created to find the deployment chain object in the datastore and add the on going create deployment as the deploymentChain.block[0] item, that brings complicated to the CreateDeployment interface
Currently, that part is addressed here: https://github.com/pipe-cd/pipe/pull/2815/files#diff-dc8d28c3b055fcad0f022406b744ce756e3be8c630b6c40aef12e026c6a58dcfR1019-R1032

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we also need to provide matchers to filter out applications for chain block in that request body, right?

Yes, we will need those matchers.

not sure what do you mean by "we can have both chain ID and the first deployment ID at the same time"

I meant by that way, we can have chain ID, deployment objects, commands in the same API. Then the first deployment can have its needed chain ID seamlessly.


What I am concerned about is how to set the chain ID for the first deployment object. In the case of doing with separate RPCs, the flow will be like this, right?

  • Piped generates chain ID locally
  • Piped requests to create a new Deployment with the generated chain ID
  • Piped requests to create a new Chain (commands, etc...)

IMO, generating the chain ID locally is not good. A Piped may send a chain ID of other projects (accidentally or intentionally).

So I think it would be better to avoid that. Another idea is reordering the above steps to

  • Piped requests to create a new Chain (commands, etc..) and chain ID is returned
  • Piped requests to create a new Deployment with the received chain ID

Copy link
Member Author

@khanhtc1202 khanhtc1202 Nov 22, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeh, in that case, we have to use the chain id to find the chain and add deployment ref to chain: we have 2 different possible cases:

  • the common case: the creating deployment come by pipedx when it's triggered by chain_sync_app command, and besides with the chain id, we also need block index to add the deployment to the right block
  • the uncommon case: the creating deployment is the first deployment of the chain, in this case, we have to add a way to separate it with others, maybe by check for matcher existed in the request body or not, or specified block_index = 0 in the request body

Piped requests to create a new Chain (commands, etc..) and chain ID is returned

One minor point for this is, at the time when we make commands chain_sync_app for applications in the chain, the first application deployment is not yet in the block[0] of the deployment chain model, if pipedx get the command and ask (via RPC) controlplane should it handle its command, we don't have any "previous" stage to determine. To avoid that, we have to ensure commands only be created after the first deployment is available as chain block[0] by making it as currently or adding matcher to the CreateDeploymentRequest as optional. The new CreateDeploymentChain rpc only creates DeploymentChain model object with its Id, ProjectId, and an empty Blocks array, nothing else. All other stub on CreateDeploymentChain should be moved to CreateDeployment rpc. I'm okay with that, but just want to confirm 👍

If you're worried about the ChainId generated in the client (piped side), we have another way to avoid that. That is generate it in the controlplane (API side) and update this req.FirstDeployment here before actually create/add it to the datastore. How do you think about that?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed by 4a4ec21

ProjectId: projectID,
Blocks: chainBlocks,
}

// Create a new deployment chain instance to control newly triggered deployment chain.
if err := a.deploymentChainStore.AddDeploymentChain(ctx, &dc); err != nil {
a.logger.Error("failed to create deployment chain", zap.Error(err))
return nil, status.Error(codes.Internal, "failed to trigger new deployment chain")
}

firstDeployment := req.FirstDeployment
firstDeployment.DeploymentChainId = dc.Id
// Trigger new deployment for the first application by store first deployment to datastore.
if err := a.deploymentStore.AddDeployment(ctx, firstDeployment); err != nil {
a.logger.Error("failed to create deployment", zap.Error(err))
return nil, status.Error(codes.Internal, "failed to trigger new deployment for the first application in chain")
}

// Make sync application command for applications of the chain.
for _, app := range apps {
cmd := model.Command{
Id: uuid.New().String(),
PipedId: app.PipedId,
ApplicationId: app.Id,
ProjectId: app.ProjectId,
Commander: dc.Id,
// TODO: Add new command type in order to separate this in chain sync command with other commands.
Type: model.Command_SYNC_APPLICATION,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we use the newly created command instead of this?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will address with a separated PR 🙏

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed my mind, lets me address it in this PR

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure.

SyncApplication: &model.Command_SyncApplication{
ApplicationId: app.Id,
SyncStrategy: model.SyncStrategy_AUTO,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ChainID and BlockIndex will be added in separate PR, right?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

addressed by 91a442b

},
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we need a way to distinguish from the normal SYNC command emitted via the web to have a better summary for its deployment.
https://github.com/pipe-cd/pipe/blob/master/pkg/app/piped/trigger/trigger.go#L259

Copy link
Member Author

@khanhtc1202 khanhtc1202 Nov 19, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are on the same page 👍 My plan is to make a new command type (such as IN_CHAIN_SYNC_APPLICATION) and filter out all commands with type prefix IN_CHAIN and handle those commands with OnChainDeterminer instead of OnCommandDeterminer. Also, the new command kind needs to have: deploymentChainId and NodeIndex at least so that the pipedx (who handles the command) will be able to update this deployment chain model its deployment state correctly. That is the thing I'm going to do with that TODO comment.

}

if err := addCommand(ctx, a.commandStore, &cmd, a.logger); err != nil {
a.logger.Error("failed to create command to trigger application in chain", zap.Error(err))
return nil, status.Error(codes.Internal, "failed to command to trigger for applications in chain")
}
}

return &pipedservice.CreateDeploymentChainResponse{}, nil
}

// validateAppBelongsToPiped checks if the given application belongs to the given piped.
// It gives back an error unless the application belongs to the piped.
func (a *PipedAPI) validateAppBelongsToPiped(ctx context.Context, appID, pipedID string) error {
Expand Down
18 changes: 18 additions & 0 deletions pkg/app/api/service/pipedservice/service.proto
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,10 @@ service PipedService {
rpc UpdateApplicationConfigurations(UpdateApplicationConfigurationsRequest) returns (UpdateApplicationConfigurationsResponse) {}
// ReportLatestUnusedApplicationConfigurations puts the latest configurations of applications that isn't registered yet.
rpc ReportUnregisteredApplicationConfigurations(ReportUnregisteredApplicationConfigurationsRequest) returns (ReportUnregisteredApplicationConfigurationsResponse) {}

// CreateDeploymentChain creates a new deployment chain object and all required commands to
// trigger deployment for applications in the chain.
rpc CreateDeploymentChain(CreateDeploymentChainRequest) returns (CreateDeploymentChainResponse) {}
}

enum ListOrder {
Expand Down Expand Up @@ -477,3 +481,17 @@ message ReportUnregisteredApplicationConfigurationsRequest {

message ReportUnregisteredApplicationConfigurationsResponse {
}

message CreateDeploymentChainRequest {
message ApplicationMatcher {
string name = 1;
string kind = 2;
map<string,string> labels = 3;
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about using the same Node as it is?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

tbh, I'm just want to rename the Node to ApplicationsFilter since those things are from the spec.postSync.chain configuration and at the time we load it from the configuration file, it's kind of filter used to find applications more than node or applications, the node word come when we create deployment chain, and applications come from users side, just to notify users that the filters written there used to find out applications for the chain.

Copy link
Member

@nghialv nghialv Nov 19, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok. I get your points.

So it is time to find some good names for all those things: name in the config, name in gRPC request, name in chain model. 🤔

How do you think about this?

// config

type DeploymentChain struct {
	Applications []ChainApplicationMatcher`json:"applications"`
	// Conditions *ChainTriggerCondition `json:"conditions,omitempty"`
}

type ChainApplicationMatcher struct {
	Name   string            `json:"name"`
	Kind   string            `json:"kind"`
	Labels map[string]string `json:"labels"`
}

// gRPC request

// use the same name with config
type CreateDeploymentChainRequest struct {
  ApplicationMatcher {
  }

  ID string
  Applications []ApplicationMatcher
}

// model in Proto format

type DeploymentChain struct {
    ID string
    Blocks []ChainBlock
}

type ChainBlock struct {
    Nodes []ChainNode
}

type ChainNode struct {
     Application *ChainApplicationRef
     Deployment *ChainDeploymentRef
     Runnable bool
}

// Just needed information to render on the web.
type ChainApplicationRef struct {
   AppID string
   AppName string
}

// Just needed information to render on the web.
// Basically we just need the "status" and ID to link to the actual one.
type ChainDeploymentRef struct {
  DeploymentID string
  ...
  CompletedAt int64
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ChainBlock or ChainGroup (Maybe we should use ChainBlock to indicate that we are using Blockchain technology in our project. 🤣 )

Copy link
Member Author

@khanhtc1202 khanhtc1202 Nov 19, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The control plane can run a simple check every time based on the deployment status to decide?

Yeh, the control-plane runs the check, but the pipedx who deploys the deployment just internally check does it should start triggering its deployment, so we need that runnable lock, I think

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I got your points. Looks nice.

Just a small thing, about where to put runnable field, if that field will be set by control-plne so I still prefer to put it in each node. By that way we still be abe to get what we want and beside that we can have more control on running order of each application. I mean controlling an application gives us more opportunity than controlling whole block.
(example, we allow apps in a block could be run serially.)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

okay, I get that. For now, it will be lock/unlock all nodes of a block at the same time, right?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

note: we may don't need that lock, since the control-plane knows when the previous block deployed successfully, one rpc call to ask control-plane whether the pipedx should start its deployment is enough 👍

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cool. 👍

pipe.model.Deployment first_deployment = 1 [(validate.rules).message.required = true];
repeated ApplicationMatcher matchers = 2;
}

message CreateDeploymentChainResponse {
}
1 change: 1 addition & 0 deletions pkg/app/piped/trigger/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"cache.go",
"deployment.go",
"deployment_chain.go",
"determiner.go",
"trigger.go",
],
Expand Down
64 changes: 16 additions & 48 deletions pkg/app/piped/trigger/deployment.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"time"

"github.com/google/uuid"
"go.uber.org/zap"

"github.com/pipe-cd/pipe/pkg/app/api/service/pipedservice"
"github.com/pipe-cd/pipe/pkg/config"
Expand All @@ -31,47 +30,14 @@ import (

func (t *Trigger) triggerDeployment(
ctx context.Context,
app *model.Application,
appCfg *config.GenericDeploymentSpec,
branch string,
commit git.Commit,
commander string,
syncStrategy model.SyncStrategy,
strategySummary string,
) (*model.Deployment, error) {

// Build deployment model to trigger.
deployment, err := buildDeployment(
app,
branch,
commit,
commander,
syncStrategy,
strategySummary,
time.Now(),
appCfg.DeploymentNotification,
)
if err != nil {
return nil, fmt.Errorf("could not initialize deployment: %w", err)
}

// Send deployment model to control-plane to trigger.
t.logger.Info(fmt.Sprintf("application %s will be triggered to sync", app.Id), zap.String("commit", commit.Hash))
_, err = t.apiClient.CreateDeployment(ctx, &pipedservice.CreateDeploymentRequest{
deployment *model.Deployment,
) error {
if _, err := t.apiClient.CreateDeployment(ctx, &pipedservice.CreateDeploymentRequest{
Deployment: deployment,
})
if err != nil {
return nil, fmt.Errorf("cound not register a new deployment to control-plane: %w", err)
}); err != nil {
return fmt.Errorf("cound not register a new deployment to control-plane: %w", err)
}

// TODO: Find a better way to ensure that the application should be updated correctly
// when the deployment was successfully triggered.
// This error is ignored because the deployment was already registered successfully.
if e := reportMostRecentlyTriggeredDeployment(ctx, t.apiClient, deployment); e != nil {
t.logger.Error("failed to report most recently triggered deployment", zap.Error(e))
}

return deployment, nil
return nil
}

func buildDeployment(
Expand All @@ -83,6 +49,7 @@ func buildDeployment(
strategySummary string,
now time.Time,
noti *config.DeploymentNotification,
deploymentChainId string,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

func parameter deploymentChainId should be deploymentChainID

https://golang.org/wiki/CodeReviewComments#initialisms

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

func parameter deploymentChainId should be deploymentChainID

https://golang.org/wiki/CodeReviewComments#initialisms

) (*model.Deployment, error) {

var commitURL string
Expand Down Expand Up @@ -125,14 +92,15 @@ func buildDeployment(
SyncStrategy: syncStrategy,
StrategySummary: strategySummary,
},
GitPath: app.GitPath,
CloudProvider: app.CloudProvider,
Labels: app.Labels,
Status: model.DeploymentStatus_DEPLOYMENT_PENDING,
StatusReason: "The deployment is waiting to be planned",
Metadata: metadata,
CreatedAt: now.Unix(),
UpdatedAt: now.Unix(),
GitPath: app.GitPath,
CloudProvider: app.CloudProvider,
Labels: app.Labels,
Status: model.DeploymentStatus_DEPLOYMENT_PENDING,
StatusReason: "The deployment is waiting to be planned",
Metadata: metadata,
CreatedAt: now.Unix(),
UpdatedAt: now.Unix(),
DeploymentChainId: deploymentChainId,
}

return deployment, nil
Expand Down
47 changes: 47 additions & 0 deletions pkg/app/piped/trigger/deployment_chain.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
// Copyright 2021 The PipeCD Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package trigger

import (
"context"
"fmt"

"github.com/pipe-cd/pipe/pkg/app/api/service/pipedservice"
"github.com/pipe-cd/pipe/pkg/config"
"github.com/pipe-cd/pipe/pkg/model"
)

func (t *Trigger) triggerDeploymentChain(
ctx context.Context,
dc *config.DeploymentChain,
firstDeployment *model.Deployment,
) error {
matchers := make([]*pipedservice.CreateDeploymentChainRequest_ApplicationMatcher, 0, len(dc.ApplicationMatchers))
for _, m := range dc.ApplicationMatchers {
matchers = append(matchers, &pipedservice.CreateDeploymentChainRequest_ApplicationMatcher{
Name: m.Name,
Kind: m.Kind,
Labels: m.Labels,
})
}

if _, err := t.apiClient.CreateDeploymentChain(ctx, &pipedservice.CreateDeploymentChainRequest{
Matchers: matchers,
FirstDeployment: firstDeployment,
}); err != nil {
return fmt.Errorf("could not create new deployment chain: %w", err)
}
return nil
}
45 changes: 42 additions & 3 deletions pkg/app/piped/trigger/trigger.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ type apiClient interface {
CreateDeployment(ctx context.Context, in *pipedservice.CreateDeploymentRequest, opts ...grpc.CallOption) (*pipedservice.CreateDeploymentResponse, error)
GetDeployment(ctx context.Context, in *pipedservice.GetDeploymentRequest, opts ...grpc.CallOption) (*pipedservice.GetDeploymentResponse, error)
ReportApplicationMostRecentDeployment(ctx context.Context, req *pipedservice.ReportApplicationMostRecentDeploymentRequest, opts ...grpc.CallOption) (*pipedservice.ReportApplicationMostRecentDeploymentResponse, error)
CreateDeploymentChain(ctx context.Context, in *pipedservice.CreateDeploymentChainRequest, opts ...grpc.CallOption) (*pipedservice.CreateDeploymentChainResponse, error)
}

type gitClient interface {
Expand Down Expand Up @@ -269,15 +270,53 @@ func (t *Trigger) checkRepoCandidates(ctx context.Context, repoID string, cs []c
strategy = model.SyncStrategy_AUTO
}

// Build deployment model and send a request to API to create a new deployment.
deployment, err := t.triggerDeployment(ctx, app, appCfg, branch, headCommit, commander, strategy, strategySummary)
// TODO: Add ability to get deployment chain id from CHAIN_SYNC_APPLICATION command.
var deploymentChainId string
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

var deploymentChainId should be deploymentChainID

https://golang.org/wiki/CodeReviewComments#initialisms

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
var deploymentChainId string
var deploymentChainID string

// Build the deployment to trigger.
deployment, err := buildDeployment(
app,
branch,
headCommit,
commander,
strategy,
strategySummary,
time.Now(),
appCfg.DeploymentNotification,
deploymentChainId,
)
if err != nil {
msg := fmt.Sprintf("failed to trigger application %s: %v", app.Id, err)
msg := fmt.Sprintf("failed to build deployment for application %s: %v", app.Id, err)
t.notifyDeploymentTriggerFailed(app, appCfg, msg, headCommit)
t.logger.Error(msg, zap.Error(err))
continue
}

// In case the triggered deployment is of application that can trigger a deployment chain
// create a new deployment chain with it's configuration.
if appCfg.PostSync != nil && appCfg.PostSync.DeploymentChain != nil {
if err := t.triggerDeploymentChain(ctx, appCfg.PostSync.DeploymentChain, deployment); err != nil {
msg := fmt.Sprintf("failed to trigger new deployment chain: %v", err)
t.notifyDeploymentTriggerFailed(app, appCfg, msg, headCommit)
t.logger.Error(msg, zap.Error(err))
continue
}
} else {
// Build deployment model and send a request to API to create a new deployment.
if err := t.triggerDeployment(ctx, deployment); err != nil {
msg := fmt.Sprintf("failed to trigger application %s: %v", app.Id, err)
t.notifyDeploymentTriggerFailed(app, appCfg, msg, headCommit)
t.logger.Error(msg, zap.Error(err))
continue
}
}

// TODO: Find a better way to ensure that the application should be updated correctly
// when the deployment was successfully triggered.
// This error is ignored because the deployment was already registered successfully.
if e := reportMostRecentlyTriggeredDeployment(ctx, t.apiClient, deployment); e != nil {
t.logger.Error("failed to report most recently triggered deployment", zap.Error(e))
}

triggered[app.Id] = struct{}{}
t.commitStore.Put(app.Id, headCommit.Hash)
t.notifyDeploymentTriggered(ctx, appCfg, deployment)
Expand Down
Loading