From 729376daef759336f97de5cc0c4b0d0bf79fed0b Mon Sep 17 00:00:00 2001
From: khanhtc1202 <khanhtc1202@gmail.com>
Date: Wed, 24 Jul 2024 18:59:32 +0700
Subject: [PATCH 1/2] Revise plugin API

Signed-off-by: khanhtc1202 <khanhtc1202@gmail.com>
---
 pkg/app/pipedv1/controller/planner.go         |  207 +---
 pkg/app/pipedv1/controller/pluginregistry.go  |   10 +-
 .../plugin/{platform => }/kubernetes/main.go  |    0
 .../kubernetes/planner/pipeline.go            |    0
 .../plugin/kubernetes/planner/server.go       |   63 +
 .../{platform => }/kubernetes/server.go       |    0
 .../platform/kubernetes/planner/server.go     |  662 -----------
 .../api/v1alpha1/{platform => }/client.go     |   18 +-
 pkg/plugin/api/v1alpha1/deployment/api.pb.go  |  880 ++++++++++++++
 .../api.pb.validate.go                        |  613 +++-------
 .../{platform => deployment}/api.proto        |   55 +-
 .../api/v1alpha1/deployment/api_grpc.pb.go    |  221 ++++
 pkg/plugin/api/v1alpha1/platform/api.pb.go    | 1009 -----------------
 .../api/v1alpha1/platform/api_grpc.pb.go      |  336 ------
 tool/codegen/codegen.sh                       |    2 +-
 15 files changed, 1367 insertions(+), 2709 deletions(-)
 rename pkg/app/pipedv1/plugin/{platform => }/kubernetes/main.go (100%)
 rename pkg/app/pipedv1/plugin/{platform => }/kubernetes/planner/pipeline.go (100%)
 create mode 100644 pkg/app/pipedv1/plugin/kubernetes/planner/server.go
 rename pkg/app/pipedv1/plugin/{platform => }/kubernetes/server.go (100%)
 delete mode 100644 pkg/app/pipedv1/plugin/platform/kubernetes/planner/server.go
 rename pkg/plugin/api/v1alpha1/{platform => }/client.go (79%)
 create mode 100644 pkg/plugin/api/v1alpha1/deployment/api.pb.go
 rename pkg/plugin/api/v1alpha1/{platform => deployment}/api.pb.validate.go (63%)
 rename pkg/plugin/api/v1alpha1/{platform => deployment}/api.proto (62%)
 create mode 100644 pkg/plugin/api/v1alpha1/deployment/api_grpc.pb.go
 delete mode 100644 pkg/plugin/api/v1alpha1/platform/api.pb.go
 delete mode 100644 pkg/plugin/api/v1alpha1/platform/api_grpc.pb.go

diff --git a/pkg/app/pipedv1/controller/planner.go b/pkg/app/pipedv1/controller/planner.go
index 49785030db..69952c7eee 100644
--- a/pkg/app/pipedv1/controller/planner.go
+++ b/pkg/app/pipedv1/controller/planner.go
@@ -16,8 +16,6 @@ package controller
 
 import (
 	"context"
-	"encoding/json"
-	"fmt"
 	"time"
 
 	"go.uber.org/atomic"
@@ -25,10 +23,8 @@ import (
 
 	"github.com/pipe-cd/pipecd/pkg/app/pipedv1/controller/controllermetrics"
 	"github.com/pipe-cd/pipecd/pkg/app/pipedv1/metadatastore"
-	"github.com/pipe-cd/pipecd/pkg/app/server/service/pipedservice"
-	"github.com/pipe-cd/pipecd/pkg/config"
 	"github.com/pipe-cd/pipecd/pkg/model"
-	"github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/platform"
+	pluginapi "github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1"
 )
 
 type planner struct {
@@ -41,7 +37,7 @@ type planner struct {
 
 	// The pluginClient is used to call pluggin that actually
 	// performs planning deployment.
-	pluginClient platform.PlatformPluginClient
+	pluginClient pluginapi.PluginClient
 
 	// The apiClient is used to report the deployment status.
 	apiClient apiClient
@@ -68,7 +64,7 @@ func newPlanner(
 	lastSuccessfulCommitHash string,
 	lastSuccessfulConfigFilename string,
 	workingDir string,
-	pluginClient platform.PlatformPluginClient,
+	pluginClient pluginapi.PluginClient,
 	apiClient apiClient,
 	notifier notifier,
 	pipedConfig []byte,
@@ -151,201 +147,4 @@ func (p *planner) Run(ctx context.Context) error {
 	}()
 
 	return nil
-
-	// in := &platform.BuildPlanRequest{
-	// 	Deployment:                   p.deployment,
-	// 	WorkingDir:                   p.workingDir,
-	// 	LastSuccessfulCommitHash:     p.lastSuccessfulCommitHash,
-	// 	LastSuccessfulConfigFileName: p.lastSuccessfulConfigFilename,
-	// 	PipedConfig:                  p.pipedConfig,
-	// }
-
-	// out, err := p.pluginClient.BuildPlan(ctx, in)
-
-	// // If the deployment was already cancelled, we ignore the plan result.
-	// select {
-	// case cmd := <-p.cancelledCh:
-	// 	if cmd != nil {
-	// 		p.doneDeploymentStatus = model.DeploymentStatus_DEPLOYMENT_CANCELLED
-	// 		desc := fmt.Sprintf("Deployment was cancelled by %s while planning", cmd.Commander)
-	// 		p.reportDeploymentCancelled(ctx, cmd.Commander, desc)
-	// 		return cmd.Report(ctx, model.CommandStatus_COMMAND_SUCCEEDED, nil, nil)
-	// 	}
-	// default:
-	// }
-
-	// if err != nil {
-	// 	p.doneDeploymentStatus = model.DeploymentStatus_DEPLOYMENT_FAILURE
-	// 	return p.reportDeploymentFailed(ctx, fmt.Sprintf("Unable to plan the deployment (%v)", err))
-	// }
-
-	// p.doneDeploymentStatus = model.DeploymentStatus_DEPLOYMENT_PLANNED
-	// return p.reportDeploymentPlanned(ctx, out.Plan)
-}
-
-// func (p *planner) reportDeploymentPlanned(ctx context.Context, out *platform.DeploymentPlan) error {
-// 	var (
-// 		err   error
-// 		retry = pipedservice.NewRetry(10)
-// 		req   = &pipedservice.ReportDeploymentPlannedRequest{
-// 			DeploymentId:              p.deployment.Id,
-// 			Summary:                   out.Summary,
-// 			StatusReason:              "The deployment has been planned",
-// 			RunningCommitHash:         p.lastSuccessfulCommitHash,
-// 			RunningConfigFilename:     p.lastSuccessfulConfigFilename,
-// 			Versions:                  out.Versions,
-// 			Stages:                    out.Stages,
-// 			DeploymentChainId:         p.deployment.DeploymentChainId,
-// 			DeploymentChainBlockIndex: p.deployment.DeploymentChainBlockIndex,
-// 		}
-// 	)
-
-// 	accounts, err := p.getMentionedAccounts(model.NotificationEventType_EVENT_DEPLOYMENT_PLANNED)
-// 	if err != nil {
-// 		p.logger.Error("failed to get the list of accounts", zap.Error(err))
-// 	}
-
-// 	defer func() {
-// 		p.notifier.Notify(model.NotificationEvent{
-// 			Type: model.NotificationEventType_EVENT_DEPLOYMENT_PLANNED,
-// 			Metadata: &model.NotificationEventDeploymentPlanned{
-// 				Deployment:        p.deployment,
-// 				Summary:           out.Summary,
-// 				MentionedAccounts: accounts,
-// 			},
-// 		})
-// 	}()
-
-// 	for retry.WaitNext(ctx) {
-// 		if _, err = p.apiClient.ReportDeploymentPlanned(ctx, req); err == nil {
-// 			return nil
-// 		}
-// 		err = fmt.Errorf("failed to report deployment status to control-plane: %v", err)
-// 	}
-
-// 	if err != nil {
-// 		p.logger.Error("failed to mark deployment to be planned", zap.Error(err))
-// 	}
-// 	return err
-// }
-
-func (p *planner) reportDeploymentFailed(ctx context.Context, reason string) error {
-	var (
-		err error
-		now = p.nowFunc()
-		req = &pipedservice.ReportDeploymentCompletedRequest{
-			DeploymentId:              p.deployment.Id,
-			Status:                    model.DeploymentStatus_DEPLOYMENT_FAILURE,
-			StatusReason:              reason,
-			StageStatuses:             nil,
-			DeploymentChainId:         p.deployment.DeploymentChainId,
-			DeploymentChainBlockIndex: p.deployment.DeploymentChainBlockIndex,
-			CompletedAt:               now.Unix(),
-		}
-		retry = pipedservice.NewRetry(10)
-	)
-
-	users, groups, err := p.getApplicationNotificationMentions(model.NotificationEventType_EVENT_DEPLOYMENT_FAILED)
-	if err != nil {
-		p.logger.Error("failed to get the list of users or groups", zap.Error(err))
-	}
-
-	defer func() {
-		p.notifier.Notify(model.NotificationEvent{
-			Type: model.NotificationEventType_EVENT_DEPLOYMENT_FAILED,
-			Metadata: &model.NotificationEventDeploymentFailed{
-				Deployment:        p.deployment,
-				Reason:            reason,
-				MentionedAccounts: users,
-				MentionedGroups:   groups,
-			},
-		})
-	}()
-
-	for retry.WaitNext(ctx) {
-		if _, err = p.apiClient.ReportDeploymentCompleted(ctx, req); err == nil {
-			return nil
-		}
-		err = fmt.Errorf("failed to report deployment status to control-plane: %v", err)
-	}
-
-	if err != nil {
-		p.logger.Error("failed to mark deployment to be failed", zap.Error(err))
-	}
-	return err
-}
-
-func (p *planner) reportDeploymentCancelled(ctx context.Context, commander, reason string) error {
-	var (
-		err error
-		now = p.nowFunc()
-		req = &pipedservice.ReportDeploymentCompletedRequest{
-			DeploymentId:              p.deployment.Id,
-			Status:                    model.DeploymentStatus_DEPLOYMENT_CANCELLED,
-			StatusReason:              reason,
-			StageStatuses:             nil,
-			DeploymentChainId:         p.deployment.DeploymentChainId,
-			DeploymentChainBlockIndex: p.deployment.DeploymentChainBlockIndex,
-			CompletedAt:               now.Unix(),
-		}
-		retry = pipedservice.NewRetry(10)
-	)
-
-	users, groups, err := p.getApplicationNotificationMentions(model.NotificationEventType_EVENT_DEPLOYMENT_CANCELLED)
-	if err != nil {
-		p.logger.Error("failed to get the list of users or groups", zap.Error(err))
-	}
-
-	defer func() {
-		p.notifier.Notify(model.NotificationEvent{
-			Type: model.NotificationEventType_EVENT_DEPLOYMENT_CANCELLED,
-			Metadata: &model.NotificationEventDeploymentCancelled{
-				Deployment:        p.deployment,
-				Commander:         commander,
-				MentionedAccounts: users,
-				MentionedGroups:   groups,
-			},
-		})
-	}()
-
-	for retry.WaitNext(ctx) {
-		if _, err = p.apiClient.ReportDeploymentCompleted(ctx, req); err == nil {
-			return nil
-		}
-		err = fmt.Errorf("failed to report deployment status to control-plane: %v", err)
-	}
-
-	if err != nil {
-		p.logger.Error("failed to mark deployment to be cancelled", zap.Error(err))
-	}
-	return err
-}
-
-func (p *planner) getMentionedUsers(event model.NotificationEventType) ([]string, error) {
-	n, ok := p.metadataStore.Shared().Get(model.MetadataKeyDeploymentNotification)
-	if !ok {
-		return []string{}, nil
-	}
-
-	var notification config.DeploymentNotification
-	if err := json.Unmarshal([]byte(n), &notification); err != nil {
-		return nil, fmt.Errorf("could not extract mentions config: %w", err)
-	}
-
-	return notification.FindSlackUsers(event), nil
-}
-
-// getApplicationNotificationMentions returns the list of users groups who should be mentioned in the notification.
-func (p *planner) getApplicationNotificationMentions(event model.NotificationEventType) ([]string, []string, error) {
-	n, ok := p.metadataStore.Shared().Get(model.MetadataKeyDeploymentNotification)
-	if !ok {
-		return []string{}, []string{}, nil
-	}
-
-	var notification config.DeploymentNotification
-	if err := json.Unmarshal([]byte(n), &notification); err != nil {
-		return nil, nil, fmt.Errorf("could not extract mentions config: %w", err)
-	}
-
-	return notification.FindSlackUsers(event), notification.FindSlackGroups(event), nil
 }
diff --git a/pkg/app/pipedv1/controller/pluginregistry.go b/pkg/app/pipedv1/controller/pluginregistry.go
index 219d53a4fd..436bfda150 100644
--- a/pkg/app/pipedv1/controller/pluginregistry.go
+++ b/pkg/app/pipedv1/controller/pluginregistry.go
@@ -24,19 +24,19 @@ import (
 	"sync"
 
 	"github.com/pipe-cd/pipecd/pkg/model"
-	"github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/platform"
+	pluginapi "github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1"
 )
 
 type PluginRegistry interface {
-	Plugin(k model.ApplicationKind) (platform.PlatformPluginClient, bool)
+	Plugin(k model.ApplicationKind) (pluginapi.PluginClient, bool)
 }
 
 type pluginRegistry struct {
-	plugins map[model.ApplicationKind]platform.PlatformPluginClient
+	plugins map[model.ApplicationKind]pluginapi.PluginClient
 	mu      sync.RWMutex
 }
 
-func (r *pluginRegistry) Plugin(k model.ApplicationKind) (platform.PlatformPluginClient, bool) {
+func (r *pluginRegistry) Plugin(k model.ApplicationKind) (pluginapi.PluginClient, bool) {
 	r.mu.RLock()
 	defer r.mu.RUnlock()
 
@@ -49,7 +49,7 @@ func (r *pluginRegistry) Plugin(k model.ApplicationKind) (platform.PlatformPlugi
 }
 
 var defaultPluginRegistry = &pluginRegistry{
-	plugins: make(map[model.ApplicationKind]platform.PlatformPluginClient),
+	plugins: make(map[model.ApplicationKind]pluginapi.PluginClient),
 }
 
 func DefaultPluginRegistry() PluginRegistry {
diff --git a/pkg/app/pipedv1/plugin/platform/kubernetes/main.go b/pkg/app/pipedv1/plugin/kubernetes/main.go
similarity index 100%
rename from pkg/app/pipedv1/plugin/platform/kubernetes/main.go
rename to pkg/app/pipedv1/plugin/kubernetes/main.go
diff --git a/pkg/app/pipedv1/plugin/platform/kubernetes/planner/pipeline.go b/pkg/app/pipedv1/plugin/kubernetes/planner/pipeline.go
similarity index 100%
rename from pkg/app/pipedv1/plugin/platform/kubernetes/planner/pipeline.go
rename to pkg/app/pipedv1/plugin/kubernetes/planner/pipeline.go
diff --git a/pkg/app/pipedv1/plugin/kubernetes/planner/server.go b/pkg/app/pipedv1/plugin/kubernetes/planner/server.go
new file mode 100644
index 0000000000..a84fbf8e78
--- /dev/null
+++ b/pkg/app/pipedv1/plugin/kubernetes/planner/server.go
@@ -0,0 +1,63 @@
+// Copyright 2024 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 planner
+
+import (
+	"context"
+	"fmt"
+
+	"github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/deployment"
+	"github.com/pipe-cd/pipecd/pkg/regexpool"
+
+	"go.uber.org/zap"
+	"google.golang.org/grpc"
+)
+
+type secretDecrypter interface {
+	Decrypt(string) (string, error)
+}
+
+type PlannerService struct {
+	deployment.UnimplementedDeploymentServiceServer
+
+	Decrypter secretDecrypter
+	RegexPool *regexpool.Pool
+	Logger    *zap.Logger
+}
+
+// Register registers all handling of this service into the specified gRPC server.
+func (a *PlannerService) Register(server *grpc.Server) {
+	deployment.RegisterDeploymentServiceServer(server, a)
+}
+
+// NewPlannerService creates a new planService.
+func NewPlannerService(
+	decrypter secretDecrypter,
+	logger *zap.Logger,
+) *PlannerService {
+	return &PlannerService{
+		Decrypter: decrypter,
+		RegexPool: regexpool.DefaultPool(),
+		Logger:    logger.Named("planner"),
+	}
+}
+
+func (ps *PlannerService) DetermineStrategy(ctx context.Context, in *deployment.DetermineStrategyRequest) (*deployment.DetermineStrategyResponse, error) {
+	return nil, fmt.Errorf("not implemented yet")
+}
+
+func (ps *PlannerService) DetermineVersions(ctx context.Context, in *deployment.DetermineVersionsRequest) (*deployment.DetermineVersionsResponse, error) {
+	return nil, fmt.Errorf("not implemented yet")
+}
diff --git a/pkg/app/pipedv1/plugin/platform/kubernetes/server.go b/pkg/app/pipedv1/plugin/kubernetes/server.go
similarity index 100%
rename from pkg/app/pipedv1/plugin/platform/kubernetes/server.go
rename to pkg/app/pipedv1/plugin/kubernetes/server.go
diff --git a/pkg/app/pipedv1/plugin/platform/kubernetes/planner/server.go b/pkg/app/pipedv1/plugin/platform/kubernetes/planner/server.go
deleted file mode 100644
index 0e334d23a7..0000000000
--- a/pkg/app/pipedv1/plugin/platform/kubernetes/planner/server.go
+++ /dev/null
@@ -1,662 +0,0 @@
-// Copyright 2024 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 planner
-
-import (
-	"context"
-	"fmt"
-	"io"
-	"os"
-	"time"
-
-	"github.com/pipe-cd/pipecd/pkg/app/pipedv1/deploysource"
-	"github.com/pipe-cd/pipecd/pkg/app/pipedv1/plugin"
-	"github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/platform"
-	"github.com/pipe-cd/pipecd/pkg/regexpool"
-
-	"go.uber.org/zap"
-	"google.golang.org/grpc"
-)
-
-type secretDecrypter interface {
-	Decrypt(string) (string, error)
-}
-
-type PlannerService struct {
-	platform.UnimplementedPlannerServiceServer
-
-	Decrypter secretDecrypter
-	RegexPool *regexpool.Pool
-	Logger    *zap.Logger
-}
-
-// Register registers all handling of this service into the specified gRPC server.
-func (a *PlannerService) Register(server *grpc.Server) {
-	platform.RegisterPlannerServiceServer(server, a)
-}
-
-// NewPlannerService creates a new planService.
-func NewPlannerService(
-	decrypter secretDecrypter,
-	logger *zap.Logger,
-) *PlannerService {
-	return &PlannerService{
-		Decrypter: decrypter,
-		RegexPool: regexpool.DefaultPool(),
-		Logger:    logger.Named("planner"),
-	}
-}
-
-// type gitClient interface {
-// 	Clone(ctx context.Context, repoID, remote, branch, destination string) (git.Repo, error)
-// 	Clean() error
-// }
-
-// type secretDecrypter interface {
-// 	Decrypt(string) (string, error)
-// }
-
-// const (
-// 	versionUnknown = "unknown"
-// )
-
-func (ps *PlannerService) DetermineStrategy(ctx context.Context, in *platform.DetermineStrategyRequest) (*platform.DetermineStrategyResponse, error) {
-	return nil, fmt.Errorf("not implemented yet")
-}
-
-func (ps *PlannerService) QuickSyncPlan(ctx context.Context, in *platform.QuickSyncPlanRequest) (*platform.QuickSyncPlanResponse, error) {
-	now := time.Now()
-
-	cloner, err := plugin.GetPlanSourceCloner(in.GetInput())
-	if err != nil {
-		return nil, err
-	}
-
-	d, err := os.MkdirTemp("", "") // TODO
-	if err != nil {
-		return nil, fmt.Errorf("failed to prepare temporary directory (%w)", err)
-	}
-	defer os.RemoveAll(d)
-
-	p := deploysource.NewProvider(
-		d,
-		cloner,
-		*in.GetInput().GetDeployment().GetGitPath(),
-		ps.Decrypter,
-	)
-
-	ds, err := p.Get(ctx, io.Discard /* TODO */)
-	if err != nil {
-		return nil, err
-	}
-
-	cfg := ds.ApplicationConfig.KubernetesApplicationSpec
-	if cfg == nil {
-		return nil, fmt.Errorf("missing KubernetesApplicationSpec in application configuration")
-	}
-
-	return &platform.QuickSyncPlanResponse{
-		Stages: buildQuickSyncPipeline(*cfg.Planner.AutoRollback, now),
-	}, nil
-}
-
-func (ps *PlannerService) PipelineSyncPlan(ctx context.Context, in *platform.PipelineSyncPlanRequest) (*platform.PipelineSyncPlanResponse, error) {
-	return nil, fmt.Errorf("not implemented yet")
-}
-
-// func (ps *PlannerService) BuildPlan(ctx context.Context, in *platform.BuildPlanRequest) (*platform.BuildPlanResponse, error) {
-// 	var (
-// 		pipedConfig     *config.PipedSpec
-// 		gitClient       gitClient
-// 		secretDecrypter secretDecrypter
-
-// 		repoCfg = config.PipedRepository{
-// 			RepoID: in.Deployment.GitPath.Repo.Id,
-// 			Remote: in.Deployment.GitPath.Repo.Remote,
-// 			Branch: in.Deployment.GitPath.Repo.Branch,
-// 		}
-// 		targetDSP  deploysource.Provider
-// 		runningDSP deploysource.Provider
-// 		out        = &platform.DeploymentPlan{}
-// 	)
-
-// 	rawCfg, err := config.DecodeYAML(in.PipedConfig)
-// 	if err != nil {
-// 		err = fmt.Errorf("failed to decode piped configuration (%v)", err)
-// 		return nil, err
-// 	}
-
-// 	pipedConfig = rawCfg.PipedSpec
-
-// 	// Initialize git client.
-// 	gitOptions := []git.Option{
-// 		git.WithUserName(pipedConfig.Git.Username),
-// 		git.WithEmail(pipedConfig.Git.Email),
-// 		git.WithLogger(ps.Logger),
-// 	}
-// 	for _, repo := range pipedConfig.GitHelmChartRepositories() {
-// 		if f := repo.SSHKeyFile; f != "" {
-// 			// Configure git client to use the specified SSH key while fetching private Helm charts.
-// 			env := fmt.Sprintf("GIT_SSH_COMMAND=ssh -i %s -o StrictHostKeyChecking=no -F /dev/null", f)
-// 			gitOptions = append(gitOptions, git.WithGitEnvForRepo(repo.GitRemote, env))
-// 		}
-// 	}
-// 	gitClient, err = git.NewClient(gitOptions...)
-// 	if err != nil {
-// 		err = fmt.Errorf("failed to create git client (%v)", err)
-// 		return nil, err
-// 	}
-// 	defer func() {
-// 		if err := gitClient.Clean(); err != nil {
-// 			ps.Logger.Error("had an error while cleaning gitClient", zap.Error(err))
-// 			return
-// 		}
-// 		ps.Logger.Info("successfully cleaned gitClient")
-// 	}()
-
-// 	// Initialize secret decrypter.
-// 	secretDecrypter, err = initializeSecretDecrypter(pipedConfig)
-// 	if err != nil {
-// 		err = fmt.Errorf("failed to initialize secret decrypter (%v)", err)
-// 		return nil, err
-// 	}
-
-// 	targetDSP = deploysource.NewProvider(
-// 		filepath.Join(in.WorkingDir, "target-deploysource"),
-// 		deploysource.NewGitSourceCloner(gitClient, repoCfg, "target", in.Deployment.Trigger.Commit.Hash),
-// 		*in.Deployment.GitPath,
-// 		secretDecrypter,
-// 	)
-
-// 	if in.LastSuccessfulCommitHash != "" {
-// 		gp := *in.Deployment.GitPath
-// 		gp.ConfigFilename = in.LastSuccessfulConfigFileName
-
-// 		runningDSP = deploysource.NewProvider(
-// 			filepath.Join(in.WorkingDir, "running-deploysource"),
-// 			deploysource.NewGitSourceCloner(gitClient, repoCfg, "running", in.LastSuccessfulCommitHash),
-// 			gp,
-// 			secretDecrypter,
-// 		)
-// 	}
-
-// 	ds, err := targetDSP.Get(ctx, io.Discard)
-// 	if err != nil {
-// 		err = fmt.Errorf("error while preparing deploy source data (%v)", err)
-// 		return nil, err
-// 	}
-// 	cfg := ds.ApplicationConfig.KubernetesApplicationSpec
-// 	if cfg == nil {
-// 		err = fmt.Errorf("missing KubernetesApplicationSpec in application configuration")
-// 		return nil, err
-// 	}
-
-// 	// TODO: get from request parameter
-// 	if cfg.Input.HelmChart != nil {
-// 		chartRepoName := cfg.Input.HelmChart.Repository
-// 		if chartRepoName != "" {
-// 			cfg.Input.HelmChart.Insecure = isInsecureChartRepository(pipedConfig, chartRepoName)
-// 		}
-// 	}
-
-// 	manifestCache := provider.AppManifestsCache{
-// 		AppID:  in.Deployment.ApplicationId,
-// 		Cache:  memorycache.NewTTLCache(ctx, time.Hour, time.Minute),
-// 		Logger: ps.Logger,
-// 	}
-
-// 	// Load previous deployed manifests and new manifests to compare.
-// 	newManifests, ok := manifestCache.Get(in.Deployment.Trigger.Commit.Hash)
-// 	if !ok {
-// 		// When the manifests were not in the cache we have to load them.
-// 		loader := provider.NewLoader(in.Deployment.ApplicationName, ds.AppDir, ds.RepoDir, in.Deployment.GitPath.ConfigFilename, cfg.Input, gitClient, ps.Logger)
-// 		newManifests, err = loader.LoadManifests(ctx)
-// 		if err != nil {
-// 			return nil, err
-// 		}
-// 		manifestCache.Put(in.Deployment.Trigger.Commit.Hash, newManifests)
-// 	}
-
-// 	if versions, e := determineVersions(newManifests); e != nil || len(versions) == 0 {
-// 		ps.Logger.Warn("unable to determine versions", zap.Error(e))
-// 		out.Versions = []*model.ArtifactVersion{
-// 			{
-// 				Kind:    model.ArtifactVersion_UNKNOWN,
-// 				Version: versionUnknown,
-// 			},
-// 		}
-// 	} else {
-// 		out.Versions = versions
-// 	}
-
-// 	autoRollback := *cfg.Input.AutoRollback
-
-// 	// In case the strategy has been decided by trigger.
-// 	// For example: user triggered the deployment via web console.
-// 	switch in.Deployment.Trigger.SyncStrategy {
-// 	case model.SyncStrategy_QUICK_SYNC:
-// 		out.SyncStrategy = model.SyncStrategy_QUICK_SYNC
-// 		out.Stages = buildQuickSyncPipeline(autoRollback, time.Now())
-// 		out.Summary = in.Deployment.Trigger.StrategySummary
-// 		return &platform.BuildPlanResponse{Plan: out}, nil
-// 	case model.SyncStrategy_PIPELINE:
-// 		if cfg.Pipeline == nil {
-// 			err = fmt.Errorf("unable to force sync with pipeline because no pipeline was specified")
-// 			return nil, err
-// 		}
-// 		out.SyncStrategy = model.SyncStrategy_PIPELINE
-// 		out.Stages = buildProgressivePipeline(cfg.Pipeline, autoRollback, time.Now())
-// 		out.Summary = in.Deployment.Trigger.StrategySummary
-// 		return &platform.BuildPlanResponse{Plan: out}, nil
-// 	}
-
-// 	// If the progressive pipeline was not configured
-// 	// we have only one choise to do is applying all manifestt.
-// 	if cfg.Pipeline == nil || len(cfg.Pipeline.Stages) == 0 {
-// 		out.SyncStrategy = model.SyncStrategy_QUICK_SYNC
-// 		out.Stages = buildQuickSyncPipeline(autoRollback, time.Now())
-// 		out.Summary = "Quick sync by applying all manifests (no pipeline was configured)"
-// 		return &platform.BuildPlanResponse{Plan: out}, nil
-// 	}
-
-// 	// Force to use pipeline when the alwaysUsePipeline field was configured.
-// 	if cfg.Planner.AlwaysUsePipeline {
-// 		out.SyncStrategy = model.SyncStrategy_PIPELINE
-// 		out.Stages = buildProgressivePipeline(cfg.Pipeline, autoRollback, time.Now())
-// 		out.Summary = "Sync with the specified pipeline (alwaysUsePipeline was set)"
-// 		return &platform.BuildPlanResponse{Plan: out}, nil
-// 	}
-
-// 	// This deployment is triggered by a commit with the intent to perform pipeline.
-// 	// Commit Matcher will be ignored when triggered by a command.
-// 	if p := cfg.CommitMatcher.Pipeline; p != "" && in.Deployment.Trigger.Commander == "" {
-// 		pipelineRegex, err := ps.RegexPool.Get(p)
-// 		if err != nil {
-// 			err = fmt.Errorf("failed to compile commitMatcher.pipeline(%s): %w", p, err)
-// 			return &platform.BuildPlanResponse{Plan: out}, err
-// 		}
-// 		if pipelineRegex.MatchString(in.Deployment.Trigger.Commit.Message) {
-// 			out.SyncStrategy = model.SyncStrategy_PIPELINE
-// 			out.Stages = buildProgressivePipeline(cfg.Pipeline, autoRollback, time.Now())
-// 			out.Summary = fmt.Sprintf("Sync progressively because the commit message was matching %q", p)
-// 			return &platform.BuildPlanResponse{Plan: out}, err
-// 		}
-// 	}
-
-// 	// This deployment is triggered by a commit with the intent to synchronize.
-// 	// Commit Matcher will be ignored when triggered by a command.
-// 	if s := cfg.CommitMatcher.QuickSync; s != "" && in.Deployment.Trigger.Commander == "" {
-// 		syncRegex, err := ps.RegexPool.Get(s)
-// 		if err != nil {
-// 			err = fmt.Errorf("failed to compile commitMatcher.sync(%s): %w", s, err)
-// 			return &platform.BuildPlanResponse{Plan: out}, err
-// 		}
-// 		if syncRegex.MatchString(in.Deployment.Trigger.Commit.Message) {
-// 			out.SyncStrategy = model.SyncStrategy_QUICK_SYNC
-// 			out.Stages = buildQuickSyncPipeline(autoRollback, time.Now())
-// 			out.Summary = fmt.Sprintf("Quick sync by applying all manifests because the commit message was matching %q", s)
-// 			return &platform.BuildPlanResponse{Plan: out}, nil
-// 		}
-// 	}
-
-// 	// This is the first time to deploy this application
-// 	// or it was unable to retrieve that value.
-// 	// We just apply all manifests.
-// 	if in.LastSuccessfulCommitHash == "" {
-// 		out.SyncStrategy = model.SyncStrategy_QUICK_SYNC
-// 		out.Stages = buildQuickSyncPipeline(autoRollback, time.Now())
-// 		out.Summary = "Quick sync by applying all manifests because it seems this is the first deployment"
-// 		return &platform.BuildPlanResponse{Plan: out}, nil
-// 	}
-
-// 	// Load manifests of the previously applied commit.
-// 	oldManifests, ok := manifestCache.Get(in.LastSuccessfulCommitHash)
-// 	if !ok {
-// 		// When the manifests were not in the cache we have to load them.
-// 		var runningDs *deploysource.DeploySource
-// 		runningDs, err = runningDSP.Get(ctx, io.Discard)
-// 		if err != nil {
-// 			err = fmt.Errorf("failed to prepare the running deploy source data (%v)", err)
-// 			return &platform.BuildPlanResponse{Plan: out}, err
-// 		}
-
-// 		runningCfg := runningDs.ApplicationConfig.KubernetesApplicationSpec
-// 		if runningCfg == nil {
-// 			err = fmt.Errorf("unable to find the running configuration (%v)", err)
-// 			return &platform.BuildPlanResponse{Plan: out}, err
-// 		}
-// 		loader := provider.NewLoader(in.Deployment.ApplicationName, runningDs.AppDir, runningDs.RepoDir, in.Deployment.GitPath.ConfigFilename, runningCfg.Input, gitClient, ps.Logger)
-// 		oldManifests, err = loader.LoadManifests(ctx)
-// 		if err != nil {
-// 			err = fmt.Errorf("failed to load previously deployed manifests: %w", err)
-// 			return &platform.BuildPlanResponse{Plan: out}, err
-// 		}
-// 		manifestCache.Put(in.LastSuccessfulCommitHash, oldManifests)
-// 	}
-
-// 	progressive, desc := decideStrategy(oldManifests, newManifests, cfg.Workloads, ps.Logger)
-// 	out.Summary = desc
-
-// 	if progressive {
-// 		out.SyncStrategy = model.SyncStrategy_PIPELINE
-// 		out.Stages = buildProgressivePipeline(cfg.Pipeline, autoRollback, time.Now())
-// 		return &platform.BuildPlanResponse{Plan: out}, err
-// 	}
-
-// 	out.SyncStrategy = model.SyncStrategy_QUICK_SYNC
-// 	out.Stages = buildQuickSyncPipeline(autoRollback, time.Now())
-// 	return &platform.BuildPlanResponse{Plan: out}, err
-// }
-
-// // First up, checks to see if the workload's `spec.template` has been changed,
-// // and then checks if the configmap/secret's data.
-// func decideStrategy(olds, news []provider.Manifest, workloadRefs []config.K8sResourceReference, logger *zap.Logger) (progressive bool, desc string) {
-// 	oldWorkloads := findWorkloadManifests(olds, workloadRefs)
-// 	if len(oldWorkloads) == 0 {
-// 		desc = "Quick sync by applying all manifests because it was unable to find the currently running workloads"
-// 		return
-// 	}
-// 	newWorkloads := findWorkloadManifests(news, workloadRefs)
-// 	if len(newWorkloads) == 0 {
-// 		desc = "Quick sync by applying all manifests because it was unable to find workloads in the new manifests"
-// 		return
-// 	}
-
-// 	workloads := findUpdatedWorkloads(oldWorkloads, newWorkloads)
-// 	diffs := make(map[provider.ResourceKey]diff.Nodes, len(workloads))
-
-// 	for _, w := range workloads {
-// 		// If the workload's pod template was touched
-// 		// do progressive deployment with the specified pipeline.
-// 		diffResult, err := provider.Diff(w.old, w.new, logger)
-// 		if err != nil {
-// 			progressive = true
-// 			desc = fmt.Sprintf("Sync progressively due to an error while calculating the diff (%v)", err)
-// 			return
-// 		}
-// 		diffNodes := diffResult.Nodes()
-// 		diffs[w.new.Key] = diffNodes
-
-// 		templateDiffs := diffNodes.FindByPrefix("spec.template")
-// 		if len(templateDiffs) > 0 {
-// 			progressive = true
-
-// 			if msg, changed := checkImageChange(templateDiffs); changed {
-// 				desc = msg
-// 				return
-// 			}
-
-// 			desc = fmt.Sprintf("Sync progressively because pod template of workload %s was changed", w.new.Key.Name)
-// 			return
-// 		}
-// 	}
-
-// 	// If the config/secret was touched, we also need to do progressive
-// 	// deployment to check run with the new config/secret content.
-// 	oldConfigs := findConfigs(olds)
-// 	newConfigs := findConfigs(news)
-// 	if len(oldConfigs) > len(newConfigs) {
-// 		progressive = true
-// 		desc = fmt.Sprintf("Sync progressively because %d configmap/secret deleted", len(oldConfigs)-len(newConfigs))
-// 		return
-// 	}
-// 	if len(oldConfigs) < len(newConfigs) {
-// 		progressive = true
-// 		desc = fmt.Sprintf("Sync progressively because new %d configmap/secret added", len(newConfigs)-len(oldConfigs))
-// 		return
-// 	}
-// 	for k, oc := range oldConfigs {
-// 		nc, ok := newConfigs[k]
-// 		if !ok {
-// 			progressive = true
-// 			desc = fmt.Sprintf("Sync progressively because %s %s was deleted", oc.Key.Kind, oc.Key.Name)
-// 			return
-// 		}
-// 		result, err := provider.Diff(oc, nc, logger)
-// 		if err != nil {
-// 			progressive = true
-// 			desc = fmt.Sprintf("Sync progressively due to an error while calculating the diff (%v)", err)
-// 			return
-// 		}
-// 		if result.HasDiff() {
-// 			progressive = true
-// 			desc = fmt.Sprintf("Sync progressively because %s %s was updated", oc.Key.Kind, oc.Key.Name)
-// 			return
-// 		}
-// 	}
-
-// 	// Check if this is a scaling commit.
-// 	scales := make([]string, 0, len(diffs))
-// 	for k, d := range diffs {
-// 		if before, after, changed := checkReplicasChange(d); changed {
-// 			scales = append(scales, fmt.Sprintf("%s/%s from %s to %s", k.Kind, k.Name, before, after))
-// 		}
-
-// 	}
-// 	sort.Strings(scales)
-// 	if len(scales) > 0 {
-// 		desc = fmt.Sprintf("Quick sync to scale %s", strings.Join(scales, ", "))
-// 		return
-// 	}
-
-// 	desc = "Quick sync by applying all manifests"
-// 	return
-// }
-
-// func initializeSecretDecrypter(cfg *config.PipedSpec) (crypto.Decrypter, error) {
-// 	sm := cfg.SecretManagement
-// 	if sm == nil {
-// 		return nil, nil
-// 	}
-
-// 	switch sm.Type {
-// 	case model.SecretManagementTypeNone:
-// 		return nil, nil
-
-// 	case model.SecretManagementTypeKeyPair:
-// 		key, err := sm.KeyPair.LoadPrivateKey()
-// 		if err != nil {
-// 			return nil, err
-// 		}
-// 		decrypter, err := crypto.NewHybridDecrypter(key)
-// 		if err != nil {
-// 			return nil, fmt.Errorf("failed to initialize decrypter (%w)", err)
-// 		}
-// 		return decrypter, nil
-
-// 	case model.SecretManagementTypeGCPKMS:
-// 		return nil, fmt.Errorf("type %q is not implemented yet", sm.Type.String())
-
-// 	case model.SecretManagementTypeAWSKMS:
-// 		return nil, fmt.Errorf("type %q is not implemented yet", sm.Type.String())
-
-// 	default:
-// 		return nil, fmt.Errorf("unsupported secret management type: %s", sm.Type.String())
-// 	}
-// }
-
-// func isInsecureChartRepository(cfg *config.PipedSpec, name string) bool {
-// 	for _, cr := range cfg.ChartRepositories {
-// 		if cr.Name == name {
-// 			return cr.Insecure
-// 		}
-// 	}
-// 	return false
-// }
-
-// func findWorkloadManifests(manifests []provider.Manifest, refs []config.K8sResourceReference) []provider.Manifest {
-// 	if len(refs) == 0 {
-// 		return findManifests(provider.KindDeployment, "", manifests)
-// 	}
-
-// 	workloads := make([]provider.Manifest, 0)
-// 	for _, ref := range refs {
-// 		kind := provider.KindDeployment
-// 		if ref.Kind != "" {
-// 			kind = ref.Kind
-// 		}
-// 		ms := findManifests(kind, ref.Name, manifests)
-// 		workloads = append(workloads, ms...)
-// 	}
-// 	return workloads
-// }
-
-// func findManifests(kind, name string, manifests []provider.Manifest) []provider.Manifest {
-// 	out := make([]provider.Manifest, 0, len(manifests))
-// 	for _, m := range manifests {
-// 		if m.Key.Kind != kind {
-// 			continue
-// 		}
-// 		if name != "" && m.Key.Name != name {
-// 			continue
-// 		}
-// 		out = append(out, m)
-// 	}
-// 	return out
-// }
-
-// type workloadPair struct {
-// 	old provider.Manifest
-// 	new provider.Manifest
-// }
-
-// func findUpdatedWorkloads(olds, news []provider.Manifest) []workloadPair {
-// 	pairs := make([]workloadPair, 0)
-// 	oldMap := make(map[provider.ResourceKey]provider.Manifest, len(olds))
-// 	nomalizeKey := func(k provider.ResourceKey) provider.ResourceKey {
-// 		// Ignoring APIVersion because user can upgrade to the new APIVersion for the same workload.
-// 		k.APIVersion = ""
-// 		if k.Namespace == provider.DefaultNamespace {
-// 			k.Namespace = ""
-// 		}
-// 		return k
-// 	}
-// 	for _, m := range olds {
-// 		key := nomalizeKey(m.Key)
-// 		oldMap[key] = m
-// 	}
-// 	for _, n := range news {
-// 		key := nomalizeKey(n.Key)
-// 		if o, ok := oldMap[key]; ok {
-// 			pairs = append(pairs, workloadPair{
-// 				old: o,
-// 				new: n,
-// 			})
-// 		}
-// 	}
-// 	return pairs
-// }
-
-// func findConfigs(manifests []provider.Manifest) map[provider.ResourceKey]provider.Manifest {
-// 	configs := make(map[provider.ResourceKey]provider.Manifest)
-// 	for _, m := range manifests {
-// 		if m.Key.IsConfigMap() {
-// 			configs[m.Key] = m
-// 		}
-// 		if m.Key.IsSecret() {
-// 			configs[m.Key] = m
-// 		}
-// 	}
-// 	return configs
-// }
-
-// func checkImageChange(ns diff.Nodes) (string, bool) {
-// 	const containerImageQuery = `^spec\.template\.spec\.containers\.\d+.image$`
-// 	nodes, _ := ns.Find(containerImageQuery)
-// 	if len(nodes) == 0 {
-// 		return "", false
-// 	}
-
-// 	images := make([]string, 0, len(ns))
-// 	for _, n := range nodes {
-// 		beforeImg := parseContainerImage(n.StringX())
-// 		afterImg := parseContainerImage(n.StringY())
-
-// 		if beforeImg.name == afterImg.name {
-// 			images = append(images, fmt.Sprintf("image %s from %s to %s", beforeImg.name, beforeImg.tag, afterImg.tag))
-// 		} else {
-// 			images = append(images, fmt.Sprintf("image %s:%s to %s:%s", beforeImg.name, beforeImg.tag, afterImg.name, afterImg.tag))
-// 		}
-// 	}
-// 	desc := fmt.Sprintf("Sync progressively because of updating %s", strings.Join(images, ", "))
-// 	return desc, true
-// }
-
-// func checkReplicasChange(ns diff.Nodes) (before, after string, changed bool) {
-// 	const replicasQuery = `^spec\.replicas$`
-// 	node, err := ns.FindOne(replicasQuery)
-// 	if err != nil {
-// 		return
-// 	}
-
-// 	before = node.StringX()
-// 	after = node.StringY()
-// 	changed = true
-// 	return
-// }
-
-// type containerImage struct {
-// 	name string
-// 	tag  string
-// }
-
-// func parseContainerImage(image string) (img containerImage) {
-// 	parts := strings.Split(image, ":")
-// 	if len(parts) == 2 {
-// 		img.tag = parts[1]
-// 	}
-// 	paths := strings.Split(parts[0], "/")
-// 	img.name = paths[len(paths)-1]
-// 	return
-// }
-
-// // determineVersions decides artifact versions of an application.
-// // It finds all container images that are being specified in the workload manifests then returns their names, version numbers, and urls.
-// func determineVersions(manifests []provider.Manifest) ([]*model.ArtifactVersion, error) {
-// 	imageMap := map[string]struct{}{}
-// 	for _, m := range manifests {
-// 		// TODO: Determine container image version from other workload kinds such as StatefulSet, Pod, Daemon, CronJob...
-// 		if !m.Key.IsDeployment() {
-// 			continue
-// 		}
-// 		data, err := m.MarshalJSON()
-// 		if err != nil {
-// 			return nil, err
-// 		}
-// 		var d resource.Deployment
-// 		if err := json.Unmarshal(data, &d); err != nil {
-// 			return nil, err
-// 		}
-
-// 		containers := d.Spec.Template.Spec.Containers
-// 		// Remove duplicate images on multiple manifests.
-// 		for _, c := range containers {
-// 			imageMap[c.Image] = struct{}{}
-// 		}
-// 	}
-
-// 	versions := make([]*model.ArtifactVersion, 0, len(imageMap))
-// 	for i := range imageMap {
-// 		image := parseContainerImage(i)
-// 		versions = append(versions, &model.ArtifactVersion{
-// 			Kind:    model.ArtifactVersion_CONTAINER_IMAGE,
-// 			Version: image.tag,
-// 			Name:    image.name,
-// 			Url:     i,
-// 		})
-// 	}
-
-// 	return versions, nil
-// }
diff --git a/pkg/plugin/api/v1alpha1/platform/client.go b/pkg/plugin/api/v1alpha1/client.go
similarity index 79%
rename from pkg/plugin/api/v1alpha1/platform/client.go
rename to pkg/plugin/api/v1alpha1/client.go
index 80e96bced9..565218f83f 100644
--- a/pkg/plugin/api/v1alpha1/platform/client.go
+++ b/pkg/plugin/api/v1alpha1/client.go
@@ -17,38 +17,36 @@
 // The planner bases on the changes from git commits
 // then builds the deployment manifests to know the behavior of the deployment.
 // From that behavior the planner can decides which pipeline should be applied.
-package platform
+package pluginapi
 
 import (
 	"context"
 
 	"google.golang.org/grpc"
 
+	"github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/deployment"
 	"github.com/pipe-cd/pipecd/pkg/rpc/rpcclient"
 )
 
-type PlatformPluginClient interface {
-	PlannerServiceClient
-	ExecutorServiceClient
+type PluginClient interface {
+	deployment.DeploymentServiceClient
 	Close() error
 }
 
 type client struct {
-	PlannerServiceClient
-	ExecutorServiceClient
+	deployment.DeploymentServiceClient
 	conn *grpc.ClientConn
 }
 
-func NewClient(ctx context.Context, address string, opts ...rpcclient.DialOption) (PlatformPluginClient, error) {
+func NewClient(ctx context.Context, address string, opts ...rpcclient.DialOption) (PluginClient, error) {
 	conn, err := rpcclient.DialContext(ctx, address, opts...)
 	if err != nil {
 		return nil, err
 	}
 
 	return &client{
-		PlannerServiceClient:  NewPlannerServiceClient(conn),
-		ExecutorServiceClient: NewExecutorServiceClient(conn),
-		conn:                  conn,
+		DeploymentServiceClient: deployment.NewDeploymentServiceClient(conn),
+		conn:                    conn,
 	}, nil
 }
 
diff --git a/pkg/plugin/api/v1alpha1/deployment/api.pb.go b/pkg/plugin/api/v1alpha1/deployment/api.pb.go
new file mode 100644
index 0000000000..eda5e38d26
--- /dev/null
+++ b/pkg/plugin/api/v1alpha1/deployment/api.pb.go
@@ -0,0 +1,880 @@
+// Copyright 2024 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.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.27.1
+// 	protoc        v3.21.12
+// source: pkg/plugin/api/v1alpha1/deployment/api.proto
+
+package deployment
+
+import (
+	_ "github.com/envoyproxy/protoc-gen-validate/validate"
+	model "github.com/pipe-cd/pipecd/pkg/model"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+type DetermineVersionsRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Input *PlanPluginInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"`
+}
+
+func (x *DetermineVersionsRequest) Reset() {
+	*x = DetermineVersionsRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DetermineVersionsRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DetermineVersionsRequest) ProtoMessage() {}
+
+func (x *DetermineVersionsRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DetermineVersionsRequest.ProtoReflect.Descriptor instead.
+func (*DetermineVersionsRequest) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *DetermineVersionsRequest) GetInput() *PlanPluginInput {
+	if x != nil {
+		return x.Input
+	}
+	return nil
+}
+
+type DetermineVersionsResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The determined versions of the artifacts.
+	Versions []*model.ArtifactVersion `protobuf:"bytes,1,rep,name=versions,proto3" json:"versions,omitempty"`
+}
+
+func (x *DetermineVersionsResponse) Reset() {
+	*x = DetermineVersionsResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DetermineVersionsResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DetermineVersionsResponse) ProtoMessage() {}
+
+func (x *DetermineVersionsResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DetermineVersionsResponse.ProtoReflect.Descriptor instead.
+func (*DetermineVersionsResponse) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{1}
+}
+
+func (x *DetermineVersionsResponse) GetVersions() []*model.ArtifactVersion {
+	if x != nil {
+		return x.Versions
+	}
+	return nil
+}
+
+type DetermineStrategyRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Input *PlanPluginInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"`
+}
+
+func (x *DetermineStrategyRequest) Reset() {
+	*x = DetermineStrategyRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DetermineStrategyRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DetermineStrategyRequest) ProtoMessage() {}
+
+func (x *DetermineStrategyRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DetermineStrategyRequest.ProtoReflect.Descriptor instead.
+func (*DetermineStrategyRequest) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{2}
+}
+
+func (x *DetermineStrategyRequest) GetInput() *PlanPluginInput {
+	if x != nil {
+		return x.Input
+	}
+	return nil
+}
+
+type DetermineStrategyResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The determined sync strategy.
+	SyncStrategy model.SyncStrategy `protobuf:"varint,1,opt,name=sync_strategy,json=syncStrategy,proto3,enum=model.SyncStrategy" json:"sync_strategy,omitempty"`
+	// Text summary of the determined strategy.
+	Summary string `protobuf:"bytes,2,opt,name=summary,proto3" json:"summary,omitempty"`
+}
+
+func (x *DetermineStrategyResponse) Reset() {
+	*x = DetermineStrategyResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DetermineStrategyResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DetermineStrategyResponse) ProtoMessage() {}
+
+func (x *DetermineStrategyResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DetermineStrategyResponse.ProtoReflect.Descriptor instead.
+func (*DetermineStrategyResponse) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{3}
+}
+
+func (x *DetermineStrategyResponse) GetSyncStrategy() model.SyncStrategy {
+	if x != nil {
+		return x.SyncStrategy
+	}
+	return model.SyncStrategy(0)
+}
+
+func (x *DetermineStrategyResponse) GetSummary() string {
+	if x != nil {
+		return x.Summary
+	}
+	return ""
+}
+
+type BuildStagesRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Stages []*BuildStagesRequest_StageConfig `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"`
+}
+
+func (x *BuildStagesRequest) Reset() {
+	*x = BuildStagesRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *BuildStagesRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*BuildStagesRequest) ProtoMessage() {}
+
+func (x *BuildStagesRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use BuildStagesRequest.ProtoReflect.Descriptor instead.
+func (*BuildStagesRequest) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{4}
+}
+
+func (x *BuildStagesRequest) GetStages() []*BuildStagesRequest_StageConfig {
+	if x != nil {
+		return x.Stages
+	}
+	return nil
+}
+
+type BuildStagesResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The built stages of the deployment pipeline.
+	Stages []*model.PipelineStage `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"`
+}
+
+func (x *BuildStagesResponse) Reset() {
+	*x = BuildStagesResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *BuildStagesResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*BuildStagesResponse) ProtoMessage() {}
+
+func (x *BuildStagesResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use BuildStagesResponse.ProtoReflect.Descriptor instead.
+func (*BuildStagesResponse) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{5}
+}
+
+func (x *BuildStagesResponse) GetStages() []*model.PipelineStage {
+	if x != nil {
+		return x.Stages
+	}
+	return nil
+}
+
+type FetchDefinedStagesRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *FetchDefinedStagesRequest) Reset() {
+	*x = FetchDefinedStagesRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FetchDefinedStagesRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FetchDefinedStagesRequest) ProtoMessage() {}
+
+func (x *FetchDefinedStagesRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FetchDefinedStagesRequest.ProtoReflect.Descriptor instead.
+func (*FetchDefinedStagesRequest) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{6}
+}
+
+type FetchDefinedStagesResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The defined stages supported by the plugin.
+	Stages []string `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"`
+}
+
+func (x *FetchDefinedStagesResponse) Reset() {
+	*x = FetchDefinedStagesResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FetchDefinedStagesResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FetchDefinedStagesResponse) ProtoMessage() {}
+
+func (x *FetchDefinedStagesResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FetchDefinedStagesResponse.ProtoReflect.Descriptor instead.
+func (*FetchDefinedStagesResponse) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{7}
+}
+
+func (x *FetchDefinedStagesResponse) GetStages() []string {
+	if x != nil {
+		return x.Stages
+	}
+	return nil
+}
+
+type PlanPluginInput struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The deployment to build a plan for.
+	Deployment *model.Deployment `protobuf:"bytes,1,opt,name=deployment,proto3" json:"deployment,omitempty"`
+	// The remote URL of the deployment source, where plugin can find the deployments sources (manifests).
+	SourceRemoteUrl string `protobuf:"bytes,2,opt,name=source_remote_url,json=sourceRemoteUrl,proto3" json:"source_remote_url,omitempty"`
+	// Last successful commit hash and config file name.
+	// Use to build deployment source object for last successful deployment.
+	LastSuccessfulCommitHash     string `protobuf:"bytes,3,opt,name=last_successful_commit_hash,json=lastSuccessfulCommitHash,proto3" json:"last_successful_commit_hash,omitempty"`
+	LastSuccessfulConfigFileName string `protobuf:"bytes,4,opt,name=last_successful_config_file_name,json=lastSuccessfulConfigFileName,proto3" json:"last_successful_config_file_name,omitempty"`
+	// The configuration of plugin that handles the deployment.
+	PluginConfig []byte `protobuf:"bytes,5,opt,name=plugin_config,json=pluginConfig,proto3" json:"plugin_config,omitempty"`
+}
+
+func (x *PlanPluginInput) Reset() {
+	*x = PlanPluginInput{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PlanPluginInput) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PlanPluginInput) ProtoMessage() {}
+
+func (x *PlanPluginInput) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PlanPluginInput.ProtoReflect.Descriptor instead.
+func (*PlanPluginInput) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{8}
+}
+
+func (x *PlanPluginInput) GetDeployment() *model.Deployment {
+	if x != nil {
+		return x.Deployment
+	}
+	return nil
+}
+
+func (x *PlanPluginInput) GetSourceRemoteUrl() string {
+	if x != nil {
+		return x.SourceRemoteUrl
+	}
+	return ""
+}
+
+func (x *PlanPluginInput) GetLastSuccessfulCommitHash() string {
+	if x != nil {
+		return x.LastSuccessfulCommitHash
+	}
+	return ""
+}
+
+func (x *PlanPluginInput) GetLastSuccessfulConfigFileName() string {
+	if x != nil {
+		return x.LastSuccessfulConfigFileName
+	}
+	return ""
+}
+
+func (x *PlanPluginInput) GetPluginConfig() []byte {
+	if x != nil {
+		return x.PluginConfig
+	}
+	return nil
+}
+
+type BuildStagesRequest_StageConfig struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The stage name.
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// The configuration of the stage.
+	Config []byte `protobuf:"bytes,2,opt,name=config,proto3" json:"config,omitempty"`
+}
+
+func (x *BuildStagesRequest_StageConfig) Reset() {
+	*x = BuildStagesRequest_StageConfig{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *BuildStagesRequest_StageConfig) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*BuildStagesRequest_StageConfig) ProtoMessage() {}
+
+func (x *BuildStagesRequest_StageConfig) ProtoReflect() protoreflect.Message {
+	mi := &file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use BuildStagesRequest_StageConfig.ProtoReflect.Descriptor instead.
+func (*BuildStagesRequest_StageConfig) Descriptor() ([]byte, []int) {
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP(), []int{4, 0}
+}
+
+func (x *BuildStagesRequest_StageConfig) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+func (x *BuildStagesRequest_StageConfig) GetConfig() []byte {
+	if x != nil {
+		return x.Config
+	}
+	return nil
+}
+
+var File_pkg_plugin_api_v1alpha1_deployment_api_proto protoreflect.FileDescriptor
+
+var file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDesc = []byte{
+	0x0a, 0x2c, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2f, 0x61, 0x70, 0x69,
+	0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79,
+	0x6d, 0x65, 0x6e, 0x74, 0x2f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x22,
+	0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c,
+	0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
+	0x61, 0x31, 0x1a, 0x17, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x61, 0x6c,
+	0x69, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x16, 0x70, 0x6b, 0x67,
+	0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x1a, 0x1a, 0x70, 0x6b, 0x67, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x64,
+	0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22,
+	0x6f, 0x0a, 0x18, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72, 0x73,
+	0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x53, 0x0a, 0x05, 0x69,
+	0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x67, 0x72, 0x70,
+	0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d,
+	0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e,
+	0x50, 0x6c, 0x61, 0x6e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x42,
+	0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,
+	0x22, 0x4f, 0x0a, 0x19, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72,
+	0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x32, 0x0a,
+	0x08, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32,
+	0x16, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74,
+	0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e,
+	0x73, 0x22, 0x6f, 0x0a, 0x18, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x53, 0x74,
+	0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x53, 0x0a,
+	0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x67,
+	0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f,
+	0x79, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61,
+	0x31, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75,
+	0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x05, 0x69, 0x6e, 0x70,
+	0x75, 0x74, 0x22, 0x6f, 0x0a, 0x19, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x53,
+	0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
+	0x38, 0x0a, 0x0d, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x53,
+	0x79, 0x6e, 0x63, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x0c, 0x73, 0x79, 0x6e,
+	0x63, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d,
+	0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d,
+	0x61, 0x72, 0x79, 0x22, 0xab, 0x01, 0x0a, 0x12, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x53, 0x74, 0x61,
+	0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x5a, 0x0a, 0x06, 0x73, 0x74,
+	0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x67, 0x72, 0x70,
+	0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d,
+	0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e,
+	0x42, 0x75, 0x69, 0x6c, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06,
+	0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x67, 0x65, 0x43,
+	0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6e,
+	0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69,
+	0x67, 0x22, 0x43, 0x0a, 0x13, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x67,
+	0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x06,
+	0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x22, 0x1b, 0x0a, 0x19, 0x46, 0x65, 0x74, 0x63, 0x68, 0x44,
+	0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x22, 0x34, 0x0a, 0x1a, 0x46, 0x65, 0x74, 0x63, 0x68, 0x44, 0x65, 0x66, 0x69,
+	0x6e, 0x65, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+	0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,
+	0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x22, 0xaf, 0x02, 0x0a, 0x0f, 0x50, 0x6c,
+	0x61, 0x6e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x3b, 0x0a,
+	0x0a, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x11, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79,
+	0x6d, 0x65, 0x6e, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x0a,
+	0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x33, 0x0a, 0x11, 0x73, 0x6f,
+	0x75, 0x72, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x75, 0x72, 0x6c, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x0f,
+	0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x55, 0x72, 0x6c, 0x12,
+	0x3d, 0x0a, 0x1b, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66,
+	0x75, 0x6c, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x03,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73,
+	0x73, 0x66, 0x75, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x46,
+	0x0a, 0x20, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66, 0x75,
+	0x6c, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61,
+	0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x75,
+	0x63, 0x63, 0x65, 0x73, 0x73, 0x66, 0x75, 0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x46, 0x69,
+	0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e,
+	0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x70,
+	0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x32, 0xd8, 0x04, 0x0a, 0x11,
+	0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x12, 0x95, 0x01, 0x0a, 0x12, 0x46, 0x65, 0x74, 0x63, 0x68, 0x44, 0x65, 0x66, 0x69, 0x6e,
+	0x65, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73, 0x12, 0x3d, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e,
+	0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e,
+	0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x46, 0x65,
+	0x74, 0x63, 0x68, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73,
+	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70,
+	0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74,
+	0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x46, 0x65, 0x74,
+	0x63, 0x68, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73, 0x52,
+	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x92, 0x01, 0x0a, 0x11, 0x44, 0x65,
+	0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12,
+	0x3c, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65,
+	0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c,
+	0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65,
+	0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3d, 0x2e,
+	0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c,
+	0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
+	0x61, 0x31, 0x2e, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72, 0x73,
+	0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x92,
+	0x01, 0x0a, 0x11, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x72, 0x61,
+	0x74, 0x65, 0x67, 0x79, 0x12, 0x3c, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67,
+	0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69,
+	0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d,
+	0x69, 0x6e, 0x65, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x1a, 0x3d, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e,
+	0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76,
+	0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e,
+	0x65, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+	0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x0b, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x53, 0x74, 0x61,
+	0x67, 0x65, 0x73, 0x12, 0x36, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69,
+	0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e,
+	0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x53, 0x74,
+	0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x67, 0x72,
+	0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79,
+	0x6d, 0x65, 0x6e, 0x74, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
+	0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x53, 0x74, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70,
+	0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x3e, 0x5a, 0x3c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
+	0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x2d, 0x63, 0x64, 0x2f, 0x70, 0x69, 0x70,
+	0x65, 0x63, 0x64, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2f, 0x61,
+	0x70, 0x69, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x70, 0x6c,
+	0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescOnce sync.Once
+	file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescData = file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDesc
+)
+
+func file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescGZIP() []byte {
+	file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescOnce.Do(func() {
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescData)
+	})
+	return file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDescData
+}
+
+var file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes = make([]protoimpl.MessageInfo, 10)
+var file_pkg_plugin_api_v1alpha1_deployment_api_proto_goTypes = []interface{}{
+	(*DetermineVersionsRequest)(nil),       // 0: grpc.plugin.deploymentapi.v1alpha1.DetermineVersionsRequest
+	(*DetermineVersionsResponse)(nil),      // 1: grpc.plugin.deploymentapi.v1alpha1.DetermineVersionsResponse
+	(*DetermineStrategyRequest)(nil),       // 2: grpc.plugin.deploymentapi.v1alpha1.DetermineStrategyRequest
+	(*DetermineStrategyResponse)(nil),      // 3: grpc.plugin.deploymentapi.v1alpha1.DetermineStrategyResponse
+	(*BuildStagesRequest)(nil),             // 4: grpc.plugin.deploymentapi.v1alpha1.BuildStagesRequest
+	(*BuildStagesResponse)(nil),            // 5: grpc.plugin.deploymentapi.v1alpha1.BuildStagesResponse
+	(*FetchDefinedStagesRequest)(nil),      // 6: grpc.plugin.deploymentapi.v1alpha1.FetchDefinedStagesRequest
+	(*FetchDefinedStagesResponse)(nil),     // 7: grpc.plugin.deploymentapi.v1alpha1.FetchDefinedStagesResponse
+	(*PlanPluginInput)(nil),                // 8: grpc.plugin.deploymentapi.v1alpha1.PlanPluginInput
+	(*BuildStagesRequest_StageConfig)(nil), // 9: grpc.plugin.deploymentapi.v1alpha1.BuildStagesRequest.StageConfig
+	(*model.ArtifactVersion)(nil),          // 10: model.ArtifactVersion
+	(model.SyncStrategy)(0),                // 11: model.SyncStrategy
+	(*model.PipelineStage)(nil),            // 12: model.PipelineStage
+	(*model.Deployment)(nil),               // 13: model.Deployment
+}
+var file_pkg_plugin_api_v1alpha1_deployment_api_proto_depIdxs = []int32{
+	8,  // 0: grpc.plugin.deploymentapi.v1alpha1.DetermineVersionsRequest.input:type_name -> grpc.plugin.deploymentapi.v1alpha1.PlanPluginInput
+	10, // 1: grpc.plugin.deploymentapi.v1alpha1.DetermineVersionsResponse.versions:type_name -> model.ArtifactVersion
+	8,  // 2: grpc.plugin.deploymentapi.v1alpha1.DetermineStrategyRequest.input:type_name -> grpc.plugin.deploymentapi.v1alpha1.PlanPluginInput
+	11, // 3: grpc.plugin.deploymentapi.v1alpha1.DetermineStrategyResponse.sync_strategy:type_name -> model.SyncStrategy
+	9,  // 4: grpc.plugin.deploymentapi.v1alpha1.BuildStagesRequest.stages:type_name -> grpc.plugin.deploymentapi.v1alpha1.BuildStagesRequest.StageConfig
+	12, // 5: grpc.plugin.deploymentapi.v1alpha1.BuildStagesResponse.stages:type_name -> model.PipelineStage
+	13, // 6: grpc.plugin.deploymentapi.v1alpha1.PlanPluginInput.deployment:type_name -> model.Deployment
+	6,  // 7: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.FetchDefinedStages:input_type -> grpc.plugin.deploymentapi.v1alpha1.FetchDefinedStagesRequest
+	0,  // 8: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.DetermineVersions:input_type -> grpc.plugin.deploymentapi.v1alpha1.DetermineVersionsRequest
+	2,  // 9: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.DetermineStrategy:input_type -> grpc.plugin.deploymentapi.v1alpha1.DetermineStrategyRequest
+	4,  // 10: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.BuildStages:input_type -> grpc.plugin.deploymentapi.v1alpha1.BuildStagesRequest
+	7,  // 11: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.FetchDefinedStages:output_type -> grpc.plugin.deploymentapi.v1alpha1.FetchDefinedStagesResponse
+	1,  // 12: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.DetermineVersions:output_type -> grpc.plugin.deploymentapi.v1alpha1.DetermineVersionsResponse
+	3,  // 13: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.DetermineStrategy:output_type -> grpc.plugin.deploymentapi.v1alpha1.DetermineStrategyResponse
+	5,  // 14: grpc.plugin.deploymentapi.v1alpha1.DeploymentService.BuildStages:output_type -> grpc.plugin.deploymentapi.v1alpha1.BuildStagesResponse
+	11, // [11:15] is the sub-list for method output_type
+	7,  // [7:11] is the sub-list for method input_type
+	7,  // [7:7] is the sub-list for extension type_name
+	7,  // [7:7] is the sub-list for extension extendee
+	0,  // [0:7] is the sub-list for field type_name
+}
+
+func init() { file_pkg_plugin_api_v1alpha1_deployment_api_proto_init() }
+func file_pkg_plugin_api_v1alpha1_deployment_api_proto_init() {
+	if File_pkg_plugin_api_v1alpha1_deployment_api_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DetermineVersionsRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DetermineVersionsResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DetermineStrategyRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DetermineStrategyResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*BuildStagesRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*BuildStagesResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FetchDefinedStagesRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FetchDefinedStagesResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PlanPluginInput); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*BuildStagesRequest_StageConfig); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   10,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_pkg_plugin_api_v1alpha1_deployment_api_proto_goTypes,
+		DependencyIndexes: file_pkg_plugin_api_v1alpha1_deployment_api_proto_depIdxs,
+		MessageInfos:      file_pkg_plugin_api_v1alpha1_deployment_api_proto_msgTypes,
+	}.Build()
+	File_pkg_plugin_api_v1alpha1_deployment_api_proto = out.File
+	file_pkg_plugin_api_v1alpha1_deployment_api_proto_rawDesc = nil
+	file_pkg_plugin_api_v1alpha1_deployment_api_proto_goTypes = nil
+	file_pkg_plugin_api_v1alpha1_deployment_api_proto_depIdxs = nil
+}
diff --git a/pkg/plugin/api/v1alpha1/platform/api.pb.validate.go b/pkg/plugin/api/v1alpha1/deployment/api.pb.validate.go
similarity index 63%
rename from pkg/plugin/api/v1alpha1/platform/api.pb.validate.go
rename to pkg/plugin/api/v1alpha1/deployment/api.pb.validate.go
index 7f73d2edb8..29e48d7d1e 100644
--- a/pkg/plugin/api/v1alpha1/platform/api.pb.validate.go
+++ b/pkg/plugin/api/v1alpha1/deployment/api.pb.validate.go
@@ -1,7 +1,7 @@
 // Code generated by protoc-gen-validate. DO NOT EDIT.
-// source: pkg/plugin/api/v1alpha1/platform/api.proto
+// source: pkg/plugin/api/v1alpha1/deployment/api.proto
 
-package platform
+package deployment
 
 import (
 	"bytes"
@@ -565,82 +565,76 @@ var _ interface {
 	ErrorName() string
 } = DetermineStrategyResponseValidationError{}
 
-// Validate checks the field values on QuickSyncPlanRequest with the rules
+// Validate checks the field values on BuildStagesRequest with the rules
 // defined in the proto definition for this message. If any rules are
 // violated, the first error encountered is returned, or nil if there are no violations.
-func (m *QuickSyncPlanRequest) Validate() error {
+func (m *BuildStagesRequest) Validate() error {
 	return m.validate(false)
 }
 
-// ValidateAll checks the field values on QuickSyncPlanRequest with the rules
+// ValidateAll checks the field values on BuildStagesRequest with the rules
 // defined in the proto definition for this message. If any rules are
 // violated, the result is a list of violation errors wrapped in
-// QuickSyncPlanRequestMultiError, or nil if none found.
-func (m *QuickSyncPlanRequest) ValidateAll() error {
+// BuildStagesRequestMultiError, or nil if none found.
+func (m *BuildStagesRequest) ValidateAll() error {
 	return m.validate(true)
 }
 
-func (m *QuickSyncPlanRequest) validate(all bool) error {
+func (m *BuildStagesRequest) validate(all bool) error {
 	if m == nil {
 		return nil
 	}
 
 	var errors []error
 
-	if m.GetInput() == nil {
-		err := QuickSyncPlanRequestValidationError{
-			field:  "Input",
-			reason: "value is required",
-		}
-		if !all {
-			return err
-		}
-		errors = append(errors, err)
-	}
+	for idx, item := range m.GetStages() {
+		_, _ = idx, item
 
-	if all {
-		switch v := interface{}(m.GetInput()).(type) {
-		case interface{ ValidateAll() error }:
-			if err := v.ValidateAll(); err != nil {
-				errors = append(errors, QuickSyncPlanRequestValidationError{
-					field:  "Input",
-					reason: "embedded message failed validation",
-					cause:  err,
-				})
+		if all {
+			switch v := interface{}(item).(type) {
+			case interface{ ValidateAll() error }:
+				if err := v.ValidateAll(); err != nil {
+					errors = append(errors, BuildStagesRequestValidationError{
+						field:  fmt.Sprintf("Stages[%v]", idx),
+						reason: "embedded message failed validation",
+						cause:  err,
+					})
+				}
+			case interface{ Validate() error }:
+				if err := v.Validate(); err != nil {
+					errors = append(errors, BuildStagesRequestValidationError{
+						field:  fmt.Sprintf("Stages[%v]", idx),
+						reason: "embedded message failed validation",
+						cause:  err,
+					})
+				}
 			}
-		case interface{ Validate() error }:
+		} else if v, ok := interface{}(item).(interface{ Validate() error }); ok {
 			if err := v.Validate(); err != nil {
-				errors = append(errors, QuickSyncPlanRequestValidationError{
-					field:  "Input",
+				return BuildStagesRequestValidationError{
+					field:  fmt.Sprintf("Stages[%v]", idx),
 					reason: "embedded message failed validation",
 					cause:  err,
-				})
-			}
-		}
-	} else if v, ok := interface{}(m.GetInput()).(interface{ Validate() error }); ok {
-		if err := v.Validate(); err != nil {
-			return QuickSyncPlanRequestValidationError{
-				field:  "Input",
-				reason: "embedded message failed validation",
-				cause:  err,
+				}
 			}
 		}
+
 	}
 
 	if len(errors) > 0 {
-		return QuickSyncPlanRequestMultiError(errors)
+		return BuildStagesRequestMultiError(errors)
 	}
 
 	return nil
 }
 
-// QuickSyncPlanRequestMultiError is an error wrapping multiple validation
-// errors returned by QuickSyncPlanRequest.ValidateAll() if the designated
-// constraints aren't met.
-type QuickSyncPlanRequestMultiError []error
+// BuildStagesRequestMultiError is an error wrapping multiple validation errors
+// returned by BuildStagesRequest.ValidateAll() if the designated constraints
+// aren't met.
+type BuildStagesRequestMultiError []error
 
 // Error returns a concatenation of all the error messages it wraps.
-func (m QuickSyncPlanRequestMultiError) Error() string {
+func (m BuildStagesRequestMultiError) Error() string {
 	var msgs []string
 	for _, err := range m {
 		msgs = append(msgs, err.Error())
@@ -649,11 +643,11 @@ func (m QuickSyncPlanRequestMultiError) Error() string {
 }
 
 // AllErrors returns a list of validation violation errors.
-func (m QuickSyncPlanRequestMultiError) AllErrors() []error { return m }
+func (m BuildStagesRequestMultiError) AllErrors() []error { return m }
 
-// QuickSyncPlanRequestValidationError is the validation error returned by
-// QuickSyncPlanRequest.Validate if the designated constraints aren't met.
-type QuickSyncPlanRequestValidationError struct {
+// BuildStagesRequestValidationError is the validation error returned by
+// BuildStagesRequest.Validate if the designated constraints aren't met.
+type BuildStagesRequestValidationError struct {
 	field  string
 	reason string
 	cause  error
@@ -661,24 +655,24 @@ type QuickSyncPlanRequestValidationError struct {
 }
 
 // Field function returns field value.
-func (e QuickSyncPlanRequestValidationError) Field() string { return e.field }
+func (e BuildStagesRequestValidationError) Field() string { return e.field }
 
 // Reason function returns reason value.
-func (e QuickSyncPlanRequestValidationError) Reason() string { return e.reason }
+func (e BuildStagesRequestValidationError) Reason() string { return e.reason }
 
 // Cause function returns cause value.
-func (e QuickSyncPlanRequestValidationError) Cause() error { return e.cause }
+func (e BuildStagesRequestValidationError) Cause() error { return e.cause }
 
 // Key function returns key value.
-func (e QuickSyncPlanRequestValidationError) Key() bool { return e.key }
+func (e BuildStagesRequestValidationError) Key() bool { return e.key }
 
 // ErrorName returns error name.
-func (e QuickSyncPlanRequestValidationError) ErrorName() string {
-	return "QuickSyncPlanRequestValidationError"
+func (e BuildStagesRequestValidationError) ErrorName() string {
+	return "BuildStagesRequestValidationError"
 }
 
 // Error satisfies the builtin error interface
-func (e QuickSyncPlanRequestValidationError) Error() string {
+func (e BuildStagesRequestValidationError) Error() string {
 	cause := ""
 	if e.cause != nil {
 		cause = fmt.Sprintf(" | caused by: %v", e.cause)
@@ -690,14 +684,14 @@ func (e QuickSyncPlanRequestValidationError) Error() string {
 	}
 
 	return fmt.Sprintf(
-		"invalid %sQuickSyncPlanRequest.%s: %s%s",
+		"invalid %sBuildStagesRequest.%s: %s%s",
 		key,
 		e.field,
 		e.reason,
 		cause)
 }
 
-var _ error = QuickSyncPlanRequestValidationError{}
+var _ error = BuildStagesRequestValidationError{}
 
 var _ interface {
 	Field() string
@@ -705,24 +699,24 @@ var _ interface {
 	Key() bool
 	Cause() error
 	ErrorName() string
-} = QuickSyncPlanRequestValidationError{}
+} = BuildStagesRequestValidationError{}
 
-// Validate checks the field values on QuickSyncPlanResponse with the rules
+// Validate checks the field values on BuildStagesResponse with the rules
 // defined in the proto definition for this message. If any rules are
 // violated, the first error encountered is returned, or nil if there are no violations.
-func (m *QuickSyncPlanResponse) Validate() error {
+func (m *BuildStagesResponse) Validate() error {
 	return m.validate(false)
 }
 
-// ValidateAll checks the field values on QuickSyncPlanResponse with the rules
+// ValidateAll checks the field values on BuildStagesResponse with the rules
 // defined in the proto definition for this message. If any rules are
 // violated, the result is a list of violation errors wrapped in
-// QuickSyncPlanResponseMultiError, or nil if none found.
-func (m *QuickSyncPlanResponse) ValidateAll() error {
+// BuildStagesResponseMultiError, or nil if none found.
+func (m *BuildStagesResponse) ValidateAll() error {
 	return m.validate(true)
 }
 
-func (m *QuickSyncPlanResponse) validate(all bool) error {
+func (m *BuildStagesResponse) validate(all bool) error {
 	if m == nil {
 		return nil
 	}
@@ -736,7 +730,7 @@ func (m *QuickSyncPlanResponse) validate(all bool) error {
 			switch v := interface{}(item).(type) {
 			case interface{ ValidateAll() error }:
 				if err := v.ValidateAll(); err != nil {
-					errors = append(errors, QuickSyncPlanResponseValidationError{
+					errors = append(errors, BuildStagesResponseValidationError{
 						field:  fmt.Sprintf("Stages[%v]", idx),
 						reason: "embedded message failed validation",
 						cause:  err,
@@ -744,7 +738,7 @@ func (m *QuickSyncPlanResponse) validate(all bool) error {
 				}
 			case interface{ Validate() error }:
 				if err := v.Validate(); err != nil {
-					errors = append(errors, QuickSyncPlanResponseValidationError{
+					errors = append(errors, BuildStagesResponseValidationError{
 						field:  fmt.Sprintf("Stages[%v]", idx),
 						reason: "embedded message failed validation",
 						cause:  err,
@@ -753,7 +747,7 @@ func (m *QuickSyncPlanResponse) validate(all bool) error {
 			}
 		} else if v, ok := interface{}(item).(interface{ Validate() error }); ok {
 			if err := v.Validate(); err != nil {
-				return QuickSyncPlanResponseValidationError{
+				return BuildStagesResponseValidationError{
 					field:  fmt.Sprintf("Stages[%v]", idx),
 					reason: "embedded message failed validation",
 					cause:  err,
@@ -764,19 +758,19 @@ func (m *QuickSyncPlanResponse) validate(all bool) error {
 	}
 
 	if len(errors) > 0 {
-		return QuickSyncPlanResponseMultiError(errors)
+		return BuildStagesResponseMultiError(errors)
 	}
 
 	return nil
 }
 
-// QuickSyncPlanResponseMultiError is an error wrapping multiple validation
-// errors returned by QuickSyncPlanResponse.ValidateAll() if the designated
+// BuildStagesResponseMultiError is an error wrapping multiple validation
+// errors returned by BuildStagesResponse.ValidateAll() if the designated
 // constraints aren't met.
-type QuickSyncPlanResponseMultiError []error
+type BuildStagesResponseMultiError []error
 
 // Error returns a concatenation of all the error messages it wraps.
-func (m QuickSyncPlanResponseMultiError) Error() string {
+func (m BuildStagesResponseMultiError) Error() string {
 	var msgs []string
 	for _, err := range m {
 		msgs = append(msgs, err.Error())
@@ -785,11 +779,11 @@ func (m QuickSyncPlanResponseMultiError) Error() string {
 }
 
 // AllErrors returns a list of validation violation errors.
-func (m QuickSyncPlanResponseMultiError) AllErrors() []error { return m }
+func (m BuildStagesResponseMultiError) AllErrors() []error { return m }
 
-// QuickSyncPlanResponseValidationError is the validation error returned by
-// QuickSyncPlanResponse.Validate if the designated constraints aren't met.
-type QuickSyncPlanResponseValidationError struct {
+// BuildStagesResponseValidationError is the validation error returned by
+// BuildStagesResponse.Validate if the designated constraints aren't met.
+type BuildStagesResponseValidationError struct {
 	field  string
 	reason string
 	cause  error
@@ -797,24 +791,24 @@ type QuickSyncPlanResponseValidationError struct {
 }
 
 // Field function returns field value.
-func (e QuickSyncPlanResponseValidationError) Field() string { return e.field }
+func (e BuildStagesResponseValidationError) Field() string { return e.field }
 
 // Reason function returns reason value.
-func (e QuickSyncPlanResponseValidationError) Reason() string { return e.reason }
+func (e BuildStagesResponseValidationError) Reason() string { return e.reason }
 
 // Cause function returns cause value.
-func (e QuickSyncPlanResponseValidationError) Cause() error { return e.cause }
+func (e BuildStagesResponseValidationError) Cause() error { return e.cause }
 
 // Key function returns key value.
-func (e QuickSyncPlanResponseValidationError) Key() bool { return e.key }
+func (e BuildStagesResponseValidationError) Key() bool { return e.key }
 
 // ErrorName returns error name.
-func (e QuickSyncPlanResponseValidationError) ErrorName() string {
-	return "QuickSyncPlanResponseValidationError"
+func (e BuildStagesResponseValidationError) ErrorName() string {
+	return "BuildStagesResponseValidationError"
 }
 
 // Error satisfies the builtin error interface
-func (e QuickSyncPlanResponseValidationError) Error() string {
+func (e BuildStagesResponseValidationError) Error() string {
 	cause := ""
 	if e.cause != nil {
 		cause = fmt.Sprintf(" | caused by: %v", e.cause)
@@ -826,14 +820,14 @@ func (e QuickSyncPlanResponseValidationError) Error() string {
 	}
 
 	return fmt.Sprintf(
-		"invalid %sQuickSyncPlanResponse.%s: %s%s",
+		"invalid %sBuildStagesResponse.%s: %s%s",
 		key,
 		e.field,
 		e.reason,
 		cause)
 }
 
-var _ error = QuickSyncPlanResponseValidationError{}
+var _ error = BuildStagesResponseValidationError{}
 
 var _ interface {
 	Field() string
@@ -841,84 +835,44 @@ var _ interface {
 	Key() bool
 	Cause() error
 	ErrorName() string
-} = QuickSyncPlanResponseValidationError{}
+} = BuildStagesResponseValidationError{}
 
-// Validate checks the field values on PipelineSyncPlanRequest with the rules
+// Validate checks the field values on FetchDefinedStagesRequest with the rules
 // defined in the proto definition for this message. If any rules are
 // violated, the first error encountered is returned, or nil if there are no violations.
-func (m *PipelineSyncPlanRequest) Validate() error {
+func (m *FetchDefinedStagesRequest) Validate() error {
 	return m.validate(false)
 }
 
-// ValidateAll checks the field values on PipelineSyncPlanRequest with the
+// ValidateAll checks the field values on FetchDefinedStagesRequest with the
 // rules defined in the proto definition for this message. If any rules are
 // violated, the result is a list of violation errors wrapped in
-// PipelineSyncPlanRequestMultiError, or nil if none found.
-func (m *PipelineSyncPlanRequest) ValidateAll() error {
+// FetchDefinedStagesRequestMultiError, or nil if none found.
+func (m *FetchDefinedStagesRequest) ValidateAll() error {
 	return m.validate(true)
 }
 
-func (m *PipelineSyncPlanRequest) validate(all bool) error {
+func (m *FetchDefinedStagesRequest) validate(all bool) error {
 	if m == nil {
 		return nil
 	}
 
 	var errors []error
 
-	if m.GetInput() == nil {
-		err := PipelineSyncPlanRequestValidationError{
-			field:  "Input",
-			reason: "value is required",
-		}
-		if !all {
-			return err
-		}
-		errors = append(errors, err)
-	}
-
-	if all {
-		switch v := interface{}(m.GetInput()).(type) {
-		case interface{ ValidateAll() error }:
-			if err := v.ValidateAll(); err != nil {
-				errors = append(errors, PipelineSyncPlanRequestValidationError{
-					field:  "Input",
-					reason: "embedded message failed validation",
-					cause:  err,
-				})
-			}
-		case interface{ Validate() error }:
-			if err := v.Validate(); err != nil {
-				errors = append(errors, PipelineSyncPlanRequestValidationError{
-					field:  "Input",
-					reason: "embedded message failed validation",
-					cause:  err,
-				})
-			}
-		}
-	} else if v, ok := interface{}(m.GetInput()).(interface{ Validate() error }); ok {
-		if err := v.Validate(); err != nil {
-			return PipelineSyncPlanRequestValidationError{
-				field:  "Input",
-				reason: "embedded message failed validation",
-				cause:  err,
-			}
-		}
-	}
-
 	if len(errors) > 0 {
-		return PipelineSyncPlanRequestMultiError(errors)
+		return FetchDefinedStagesRequestMultiError(errors)
 	}
 
 	return nil
 }
 
-// PipelineSyncPlanRequestMultiError is an error wrapping multiple validation
-// errors returned by PipelineSyncPlanRequest.ValidateAll() if the designated
-// constraints aren't met.
-type PipelineSyncPlanRequestMultiError []error
+// FetchDefinedStagesRequestMultiError is an error wrapping multiple validation
+// errors returned by FetchDefinedStagesRequest.ValidateAll() if the
+// designated constraints aren't met.
+type FetchDefinedStagesRequestMultiError []error
 
 // Error returns a concatenation of all the error messages it wraps.
-func (m PipelineSyncPlanRequestMultiError) Error() string {
+func (m FetchDefinedStagesRequestMultiError) Error() string {
 	var msgs []string
 	for _, err := range m {
 		msgs = append(msgs, err.Error())
@@ -927,11 +881,11 @@ func (m PipelineSyncPlanRequestMultiError) Error() string {
 }
 
 // AllErrors returns a list of validation violation errors.
-func (m PipelineSyncPlanRequestMultiError) AllErrors() []error { return m }
+func (m FetchDefinedStagesRequestMultiError) AllErrors() []error { return m }
 
-// PipelineSyncPlanRequestValidationError is the validation error returned by
-// PipelineSyncPlanRequest.Validate if the designated constraints aren't met.
-type PipelineSyncPlanRequestValidationError struct {
+// FetchDefinedStagesRequestValidationError is the validation error returned by
+// FetchDefinedStagesRequest.Validate if the designated constraints aren't met.
+type FetchDefinedStagesRequestValidationError struct {
 	field  string
 	reason string
 	cause  error
@@ -939,24 +893,24 @@ type PipelineSyncPlanRequestValidationError struct {
 }
 
 // Field function returns field value.
-func (e PipelineSyncPlanRequestValidationError) Field() string { return e.field }
+func (e FetchDefinedStagesRequestValidationError) Field() string { return e.field }
 
 // Reason function returns reason value.
-func (e PipelineSyncPlanRequestValidationError) Reason() string { return e.reason }
+func (e FetchDefinedStagesRequestValidationError) Reason() string { return e.reason }
 
 // Cause function returns cause value.
-func (e PipelineSyncPlanRequestValidationError) Cause() error { return e.cause }
+func (e FetchDefinedStagesRequestValidationError) Cause() error { return e.cause }
 
 // Key function returns key value.
-func (e PipelineSyncPlanRequestValidationError) Key() bool { return e.key }
+func (e FetchDefinedStagesRequestValidationError) Key() bool { return e.key }
 
 // ErrorName returns error name.
-func (e PipelineSyncPlanRequestValidationError) ErrorName() string {
-	return "PipelineSyncPlanRequestValidationError"
+func (e FetchDefinedStagesRequestValidationError) ErrorName() string {
+	return "FetchDefinedStagesRequestValidationError"
 }
 
 // Error satisfies the builtin error interface
-func (e PipelineSyncPlanRequestValidationError) Error() string {
+func (e FetchDefinedStagesRequestValidationError) Error() string {
 	cause := ""
 	if e.cause != nil {
 		cause = fmt.Sprintf(" | caused by: %v", e.cause)
@@ -968,14 +922,14 @@ func (e PipelineSyncPlanRequestValidationError) Error() string {
 	}
 
 	return fmt.Sprintf(
-		"invalid %sPipelineSyncPlanRequest.%s: %s%s",
+		"invalid %sFetchDefinedStagesRequest.%s: %s%s",
 		key,
 		e.field,
 		e.reason,
 		cause)
 }
 
-var _ error = PipelineSyncPlanRequestValidationError{}
+var _ error = FetchDefinedStagesRequestValidationError{}
 
 var _ interface {
 	Field() string
@@ -983,78 +937,44 @@ var _ interface {
 	Key() bool
 	Cause() error
 	ErrorName() string
-} = PipelineSyncPlanRequestValidationError{}
+} = FetchDefinedStagesRequestValidationError{}
 
-// Validate checks the field values on PipelineSyncPlanResponse with the rules
-// defined in the proto definition for this message. If any rules are
+// Validate checks the field values on FetchDefinedStagesResponse with the
+// rules defined in the proto definition for this message. If any rules are
 // violated, the first error encountered is returned, or nil if there are no violations.
-func (m *PipelineSyncPlanResponse) Validate() error {
+func (m *FetchDefinedStagesResponse) Validate() error {
 	return m.validate(false)
 }
 
-// ValidateAll checks the field values on PipelineSyncPlanResponse with the
+// ValidateAll checks the field values on FetchDefinedStagesResponse with the
 // rules defined in the proto definition for this message. If any rules are
 // violated, the result is a list of violation errors wrapped in
-// PipelineSyncPlanResponseMultiError, or nil if none found.
-func (m *PipelineSyncPlanResponse) ValidateAll() error {
+// FetchDefinedStagesResponseMultiError, or nil if none found.
+func (m *FetchDefinedStagesResponse) ValidateAll() error {
 	return m.validate(true)
 }
 
-func (m *PipelineSyncPlanResponse) validate(all bool) error {
+func (m *FetchDefinedStagesResponse) validate(all bool) error {
 	if m == nil {
 		return nil
 	}
 
 	var errors []error
 
-	for idx, item := range m.GetStages() {
-		_, _ = idx, item
-
-		if all {
-			switch v := interface{}(item).(type) {
-			case interface{ ValidateAll() error }:
-				if err := v.ValidateAll(); err != nil {
-					errors = append(errors, PipelineSyncPlanResponseValidationError{
-						field:  fmt.Sprintf("Stages[%v]", idx),
-						reason: "embedded message failed validation",
-						cause:  err,
-					})
-				}
-			case interface{ Validate() error }:
-				if err := v.Validate(); err != nil {
-					errors = append(errors, PipelineSyncPlanResponseValidationError{
-						field:  fmt.Sprintf("Stages[%v]", idx),
-						reason: "embedded message failed validation",
-						cause:  err,
-					})
-				}
-			}
-		} else if v, ok := interface{}(item).(interface{ Validate() error }); ok {
-			if err := v.Validate(); err != nil {
-				return PipelineSyncPlanResponseValidationError{
-					field:  fmt.Sprintf("Stages[%v]", idx),
-					reason: "embedded message failed validation",
-					cause:  err,
-				}
-			}
-		}
-
-	}
-
 	if len(errors) > 0 {
-		return PipelineSyncPlanResponseMultiError(errors)
+		return FetchDefinedStagesResponseMultiError(errors)
 	}
 
 	return nil
 }
 
-// PipelineSyncPlanResponseMultiError is an error wrapping multiple validation
-// errors returned by PipelineSyncPlanResponse.ValidateAll() if the designated
-// constraints aren't met.
-type PipelineSyncPlanResponseMultiError []error
+// FetchDefinedStagesResponseMultiError is an error wrapping multiple
+// validation errors returned by FetchDefinedStagesResponse.ValidateAll() if
+// the designated constraints aren't met.
+type FetchDefinedStagesResponseMultiError []error
 
 // Error returns a concatenation of all the error messages it wraps.
-func (m PipelineSyncPlanResponseMultiError) Error() string {
+func (m FetchDefinedStagesResponseMultiError) Error() string {
 	var msgs []string
 	for _, err := range m {
 		msgs = append(msgs, err.Error())
@@ -1063,11 +983,11 @@ func (m PipelineSyncPlanResponseMultiError) Error() string {
 }
 
 // AllErrors returns a list of validation violation errors.
-func (m PipelineSyncPlanResponseMultiError) AllErrors() []error { return m }
+func (m FetchDefinedStagesResponseMultiError) AllErrors() []error { return m }
 
-// PipelineSyncPlanResponseValidationError is the validation error returned by
-// PipelineSyncPlanResponse.Validate if the designated constraints aren't met.
-type PipelineSyncPlanResponseValidationError struct {
+// FetchDefinedStagesResponseValidationError is the validation error returned
+// by FetchDefinedStagesResponse.Validate if the designated constraints aren't met.
+type FetchDefinedStagesResponseValidationError struct {
 	field  string
 	reason string
 	cause  error
@@ -1075,24 +995,24 @@ type PipelineSyncPlanResponseValidationError struct {
 }
 
 // Field function returns field value.
-func (e PipelineSyncPlanResponseValidationError) Field() string { return e.field }
+func (e FetchDefinedStagesResponseValidationError) Field() string { return e.field }
 
 // Reason function returns reason value.
-func (e PipelineSyncPlanResponseValidationError) Reason() string { return e.reason }
+func (e FetchDefinedStagesResponseValidationError) Reason() string { return e.reason }
 
 // Cause function returns cause value.
-func (e PipelineSyncPlanResponseValidationError) Cause() error { return e.cause }
+func (e FetchDefinedStagesResponseValidationError) Cause() error { return e.cause }
 
 // Key function returns key value.
-func (e PipelineSyncPlanResponseValidationError) Key() bool { return e.key }
+func (e FetchDefinedStagesResponseValidationError) Key() bool { return e.key }
 
 // ErrorName returns error name.
-func (e PipelineSyncPlanResponseValidationError) ErrorName() string {
-	return "PipelineSyncPlanResponseValidationError"
+func (e FetchDefinedStagesResponseValidationError) ErrorName() string {
+	return "FetchDefinedStagesResponseValidationError"
 }
 
 // Error satisfies the builtin error interface
-func (e PipelineSyncPlanResponseValidationError) Error() string {
+func (e FetchDefinedStagesResponseValidationError) Error() string {
 	cause := ""
 	if e.cause != nil {
 		cause = fmt.Sprintf(" | caused by: %v", e.cause)
@@ -1104,14 +1024,14 @@ func (e PipelineSyncPlanResponseValidationError) Error() string {
 	}
 
 	return fmt.Sprintf(
-		"invalid %sPipelineSyncPlanResponse.%s: %s%s",
+		"invalid %sFetchDefinedStagesResponse.%s: %s%s",
 		key,
 		e.field,
 		e.reason,
 		cause)
 }
 
-var _ error = PipelineSyncPlanResponseValidationError{}
+var _ error = FetchDefinedStagesResponseValidationError{}
 
 var _ interface {
 	Field() string
@@ -1119,7 +1039,7 @@ var _ interface {
 	Key() bool
 	Cause() error
 	ErrorName() string
-} = PipelineSyncPlanResponseValidationError{}
+} = FetchDefinedStagesResponseValidationError{}
 
 // Validate checks the field values on PlanPluginInput with the rules defined
 // in the proto definition for this message. If any rules are violated, the
@@ -1278,144 +1198,46 @@ var _ interface {
 	ErrorName() string
 } = PlanPluginInputValidationError{}
 
-// Validate checks the field values on ExecuteStageRequest with the rules
-// defined in the proto definition for this message. If any rules are
+// Validate checks the field values on BuildStagesRequest_StageConfig with the
+// rules defined in the proto definition for this message. If any rules are
 // violated, the first error encountered is returned, or nil if there are no violations.
-func (m *ExecuteStageRequest) Validate() error {
+func (m *BuildStagesRequest_StageConfig) Validate() error {
 	return m.validate(false)
 }
 
-// ValidateAll checks the field values on ExecuteStageRequest with the rules
-// defined in the proto definition for this message. If any rules are
-// violated, the result is a list of violation errors wrapped in
-// ExecuteStageRequestMultiError, or nil if none found.
-func (m *ExecuteStageRequest) ValidateAll() error {
+// ValidateAll checks the field values on BuildStagesRequest_StageConfig with
+// the rules defined in the proto definition for this message. If any rules
+// are violated, the result is a list of violation errors wrapped in
+// BuildStagesRequest_StageConfigMultiError, or nil if none found.
+func (m *BuildStagesRequest_StageConfig) ValidateAll() error {
 	return m.validate(true)
 }
 
-func (m *ExecuteStageRequest) validate(all bool) error {
+func (m *BuildStagesRequest_StageConfig) validate(all bool) error {
 	if m == nil {
 		return nil
 	}
 
 	var errors []error
 
-	if m.GetStage() == nil {
-		err := ExecuteStageRequestValidationError{
-			field:  "Stage",
-			reason: "value is required",
-		}
-		if !all {
-			return err
-		}
-		errors = append(errors, err)
-	}
+	// no validation rules for Name
 
-	if all {
-		switch v := interface{}(m.GetStage()).(type) {
-		case interface{ ValidateAll() error }:
-			if err := v.ValidateAll(); err != nil {
-				errors = append(errors, ExecuteStageRequestValidationError{
-					field:  "Stage",
-					reason: "embedded message failed validation",
-					cause:  err,
-				})
-			}
-		case interface{ Validate() error }:
-			if err := v.Validate(); err != nil {
-				errors = append(errors, ExecuteStageRequestValidationError{
-					field:  "Stage",
-					reason: "embedded message failed validation",
-					cause:  err,
-				})
-			}
-		}
-	} else if v, ok := interface{}(m.GetStage()).(interface{ Validate() error }); ok {
-		if err := v.Validate(); err != nil {
-			return ExecuteStageRequestValidationError{
-				field:  "Stage",
-				reason: "embedded message failed validation",
-				cause:  err,
-			}
-		}
-	}
-
-	if len(m.GetStageConfig()) < 1 {
-		err := ExecuteStageRequestValidationError{
-			field:  "StageConfig",
-			reason: "value length must be at least 1 bytes",
-		}
-		if !all {
-			return err
-		}
-		errors = append(errors, err)
-	}
-
-	if len(m.GetPipedConfig()) < 1 {
-		err := ExecuteStageRequestValidationError{
-			field:  "PipedConfig",
-			reason: "value length must be at least 1 bytes",
-		}
-		if !all {
-			return err
-		}
-		errors = append(errors, err)
-	}
-
-	if m.GetDeployment() == nil {
-		err := ExecuteStageRequestValidationError{
-			field:  "Deployment",
-			reason: "value is required",
-		}
-		if !all {
-			return err
-		}
-		errors = append(errors, err)
-	}
-
-	if all {
-		switch v := interface{}(m.GetDeployment()).(type) {
-		case interface{ ValidateAll() error }:
-			if err := v.ValidateAll(); err != nil {
-				errors = append(errors, ExecuteStageRequestValidationError{
-					field:  "Deployment",
-					reason: "embedded message failed validation",
-					cause:  err,
-				})
-			}
-		case interface{ Validate() error }:
-			if err := v.Validate(); err != nil {
-				errors = append(errors, ExecuteStageRequestValidationError{
-					field:  "Deployment",
-					reason: "embedded message failed validation",
-					cause:  err,
-				})
-			}
-		}
-	} else if v, ok := interface{}(m.GetDeployment()).(interface{ Validate() error }); ok {
-		if err := v.Validate(); err != nil {
-			return ExecuteStageRequestValidationError{
-				field:  "Deployment",
-				reason: "embedded message failed validation",
-				cause:  err,
-			}
-		}
-	}
+	// no validation rules for Config
 
 	if len(errors) > 0 {
-		return ExecuteStageRequestMultiError(errors)
+		return BuildStagesRequest_StageConfigMultiError(errors)
 	}
 
 	return nil
 }
 
-// ExecuteStageRequestMultiError is an error wrapping multiple validation
-// errors returned by ExecuteStageRequest.ValidateAll() if the designated
-// constraints aren't met.
-type ExecuteStageRequestMultiError []error
+// BuildStagesRequest_StageConfigMultiError is an error wrapping multiple
+// validation errors returned by BuildStagesRequest_StageConfig.ValidateAll()
+// if the designated constraints aren't met.
+type BuildStagesRequest_StageConfigMultiError []error
 
 // Error returns a concatenation of all the error messages it wraps.
-func (m ExecuteStageRequestMultiError) Error() string {
+func (m BuildStagesRequest_StageConfigMultiError) Error() string {
 	var msgs []string
 	for _, err := range m {
 		msgs = append(msgs, err.Error())
@@ -1424,117 +1246,12 @@ func (m ExecuteStageRequestMultiError) Error() string {
 }
 
 // AllErrors returns a list of validation violation errors.
-func (m ExecuteStageRequestMultiError) AllErrors() []error { return m }
-
-// ExecuteStageRequestValidationError is the validation error returned by
-// ExecuteStageRequest.Validate if the designated constraints aren't met.
-type ExecuteStageRequestValidationError struct {
-	field  string
-	reason string
-	cause  error
-	key    bool
-}
-
-// Field function returns field value.
-func (e ExecuteStageRequestValidationError) Field() string { return e.field }
+func (m BuildStagesRequest_StageConfigMultiError) AllErrors() []error { return m }
 
-// Reason function returns reason value.
-func (e ExecuteStageRequestValidationError) Reason() string { return e.reason }
-
-// Cause function returns cause value.
-func (e ExecuteStageRequestValidationError) Cause() error { return e.cause }
-
-// Key function returns key value.
-func (e ExecuteStageRequestValidationError) Key() bool { return e.key }
-
-// ErrorName returns error name.
-func (e ExecuteStageRequestValidationError) ErrorName() string {
-	return "ExecuteStageRequestValidationError"
-}
-
-// Error satisfies the builtin error interface
-func (e ExecuteStageRequestValidationError) Error() string {
-	cause := ""
-	if e.cause != nil {
-		cause = fmt.Sprintf(" | caused by: %v", e.cause)
-	}
-
-	key := ""
-	if e.key {
-		key = "key for "
-	}
-
-	return fmt.Sprintf(
-		"invalid %sExecuteStageRequest.%s: %s%s",
-		key,
-		e.field,
-		e.reason,
-		cause)
-}
-
-var _ error = ExecuteStageRequestValidationError{}
-
-var _ interface {
-	Field() string
-	Reason() string
-	Key() bool
-	Cause() error
-	ErrorName() string
-} = ExecuteStageRequestValidationError{}
-
-// Validate checks the field values on ExecuteStageResponse with the rules
-// defined in the proto definition for this message. If any rules are
-// violated, the first error encountered is returned, or nil if there are no violations.
-func (m *ExecuteStageResponse) Validate() error {
-	return m.validate(false)
-}
-
-// ValidateAll checks the field values on ExecuteStageResponse with the rules
-// defined in the proto definition for this message. If any rules are
-// violated, the result is a list of violation errors wrapped in
-// ExecuteStageResponseMultiError, or nil if none found.
-func (m *ExecuteStageResponse) ValidateAll() error {
-	return m.validate(true)
-}
-
-func (m *ExecuteStageResponse) validate(all bool) error {
-	if m == nil {
-		return nil
-	}
-
-	var errors []error
-
-	// no validation rules for Status
-
-	// no validation rules for Log
-
-	if len(errors) > 0 {
-		return ExecuteStageResponseMultiError(errors)
-	}
-
-	return nil
-}
-
-// ExecuteStageResponseMultiError is an error wrapping multiple validation
-// errors returned by ExecuteStageResponse.ValidateAll() if the designated
+// BuildStagesRequest_StageConfigValidationError is the validation error
+// returned by BuildStagesRequest_StageConfig.Validate if the designated
 // constraints aren't met.
-type ExecuteStageResponseMultiError []error
-
-// Error returns a concatenation of all the error messages it wraps.
-func (m ExecuteStageResponseMultiError) Error() string {
-	var msgs []string
-	for _, err := range m {
-		msgs = append(msgs, err.Error())
-	}
-	return strings.Join(msgs, "; ")
-}
-
-// AllErrors returns a list of validation violation errors.
-func (m ExecuteStageResponseMultiError) AllErrors() []error { return m }
-
-// ExecuteStageResponseValidationError is the validation error returned by
-// ExecuteStageResponse.Validate if the designated constraints aren't met.
-type ExecuteStageResponseValidationError struct {
+type BuildStagesRequest_StageConfigValidationError struct {
 	field  string
 	reason string
 	cause  error
@@ -1542,24 +1259,24 @@ type ExecuteStageResponseValidationError struct {
 }
 
 // Field function returns field value.
-func (e ExecuteStageResponseValidationError) Field() string { return e.field }
+func (e BuildStagesRequest_StageConfigValidationError) Field() string { return e.field }
 
 // Reason function returns reason value.
-func (e ExecuteStageResponseValidationError) Reason() string { return e.reason }
+func (e BuildStagesRequest_StageConfigValidationError) Reason() string { return e.reason }
 
 // Cause function returns cause value.
-func (e ExecuteStageResponseValidationError) Cause() error { return e.cause }
+func (e BuildStagesRequest_StageConfigValidationError) Cause() error { return e.cause }
 
 // Key function returns key value.
-func (e ExecuteStageResponseValidationError) Key() bool { return e.key }
+func (e BuildStagesRequest_StageConfigValidationError) Key() bool { return e.key }
 
 // ErrorName returns error name.
-func (e ExecuteStageResponseValidationError) ErrorName() string {
-	return "ExecuteStageResponseValidationError"
+func (e BuildStagesRequest_StageConfigValidationError) ErrorName() string {
+	return "BuildStagesRequest_StageConfigValidationError"
 }
 
 // Error satisfies the builtin error interface
-func (e ExecuteStageResponseValidationError) Error() string {
+func (e BuildStagesRequest_StageConfigValidationError) Error() string {
 	cause := ""
 	if e.cause != nil {
 		cause = fmt.Sprintf(" | caused by: %v", e.cause)
@@ -1571,14 +1288,14 @@ func (e ExecuteStageResponseValidationError) Error() string {
 	}
 
 	return fmt.Sprintf(
-		"invalid %sExecuteStageResponse.%s: %s%s",
+		"invalid %sBuildStagesRequest_StageConfig.%s: %s%s",
 		key,
 		e.field,
 		e.reason,
 		cause)
 }
 
-var _ error = ExecuteStageResponseValidationError{}
+var _ error = BuildStagesRequest_StageConfigValidationError{}
 
 var _ interface {
 	Field() string
@@ -1586,4 +1303,4 @@ var _ interface {
 	Key() bool
 	Cause() error
 	ErrorName() string
-} = ExecuteStageResponseValidationError{}
+} = BuildStagesRequest_StageConfigValidationError{}
diff --git a/pkg/plugin/api/v1alpha1/platform/api.proto b/pkg/plugin/api/v1alpha1/deployment/api.proto
similarity index 62%
rename from pkg/plugin/api/v1alpha1/platform/api.proto
rename to pkg/plugin/api/v1alpha1/deployment/api.proto
index 42431c6dcc..af8926237d 100644
--- a/pkg/plugin/api/v1alpha1/platform/api.proto
+++ b/pkg/plugin/api/v1alpha1/deployment/api.proto
@@ -14,23 +14,23 @@
 
 syntax = "proto3";
 
-package grpc.plugin.platformapi.v1alpha1;
-option go_package = "github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/platform";
+package grpc.plugin.deploymentapi.v1alpha1;
+option go_package = "github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/deployment";
 
 import "validate/validate.proto";
 import "pkg/model/common.proto";
 import "pkg/model/deployment.proto";
 
 // PlannerService defines the public APIs for remote planners.
-service PlannerService {
+service DeploymentService {
+    // FetchDefinedStages fetches the defined stages' name which are supported by the plugin.
+    rpc FetchDefinedStages(FetchDefinedStagesRequest) returns (FetchDefinedStagesResponse) {}
     // DetermineVersions determines which versions of the artifacts will be used for the given deployment.
     rpc DetermineVersions(DetermineVersionsRequest) returns (DetermineVersionsResponse) {}
     // DetermineStrategy determines which strategy should be used for the given deployment.
     rpc DetermineStrategy(DetermineStrategyRequest) returns (DetermineStrategyResponse) {}
-    // QuickSyncPlan builds plan for the given deployment using quick sync strategy.
-    rpc QuickSyncPlan(QuickSyncPlanRequest) returns (QuickSyncPlanResponse) {}
-    // PipelineSyncPlan builds plan for the given deployment using pipeline sync strategy.
-    rpc PipelineSyncPlan(PipelineSyncPlanRequest) returns (PipelineSyncPlanResponse) {}
+    // BuildStages builds the deployment pipeline stages.
+    rpc BuildStages(BuildStagesRequest) returns (BuildStagesResponse) {}
 }
 
 message DetermineVersionsRequest {
@@ -53,22 +53,26 @@ message DetermineStrategyResponse {
     string summary = 2;
 }
 
-message QuickSyncPlanRequest {
-    PlanPluginInput input = 1 [(validate.rules).message.required = true];
+message BuildStagesRequest {
+    message StageConfig {
+        // The stage name.
+        string name = 1;
+        // The configuration of the stage.
+        bytes config = 2;
+    }
+    repeated StageConfig stages = 1;
 }
 
-message QuickSyncPlanResponse {
-    // Stages of deployment pipeline under quick sync strategy.
+message BuildStagesResponse {
+    // The built stages of the deployment pipeline.
     repeated model.PipelineStage stages = 1;
 }
 
-message PipelineSyncPlanRequest {
-    PlanPluginInput input = 1 [(validate.rules).message.required = true];
-}
+message FetchDefinedStagesRequest {}
 
-message PipelineSyncPlanResponse {
-    // Stages of deployment pipeline under pipeline sync strategy.
-    repeated model.PipelineStage stages = 1;
+message FetchDefinedStagesResponse {
+    // The defined stages supported by the plugin.
+    repeated string stages = 1;
 }
 
 message PlanPluginInput {
@@ -83,20 +87,3 @@ message PlanPluginInput {
     // The configuration of plugin that handles the deployment.
     bytes plugin_config = 5;
 }
-
-service ExecutorService {
-    // Execute executes the given stage of the deployment plan.
-    rpc ExecuteStage(ExecuteStageRequest) returns (stream ExecuteStageResponse) {}
-}
-
-message ExecuteStageRequest {
-    model.PipelineStage stage = 1 [(validate.rules).message.required = true];
-    bytes stage_config = 2 [(validate.rules).bytes.min_len = 1];
-    bytes piped_config = 3 [(validate.rules).bytes.min_len = 1];
-    model.Deployment deployment = 4 [(validate.rules).message.required = true];
-}
-
-message ExecuteStageResponse {
-    model.StageStatus status = 1;
-    string log = 2;
-}
diff --git a/pkg/plugin/api/v1alpha1/deployment/api_grpc.pb.go b/pkg/plugin/api/v1alpha1/deployment/api_grpc.pb.go
new file mode 100644
index 0000000000..1c621831e3
--- /dev/null
+++ b/pkg/plugin/api/v1alpha1/deployment/api_grpc.pb.go
@@ -0,0 +1,221 @@
+// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
+// versions:
+// - protoc-gen-go-grpc v1.2.0
+// - protoc             v3.21.12
+// source: pkg/plugin/api/v1alpha1/deployment/api.proto
+
+package deployment
+
+import (
+	context "context"
+	grpc "google.golang.org/grpc"
+	codes "google.golang.org/grpc/codes"
+	status "google.golang.org/grpc/status"
+)
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+// Requires gRPC-Go v1.32.0 or later.
+const _ = grpc.SupportPackageIsVersion7
+
+// DeploymentServiceClient is the client API for DeploymentService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
+type DeploymentServiceClient interface {
+	// FetchDefinedStages fetches the defined stages' name which are supported by the plugin.
+	FetchDefinedStages(ctx context.Context, in *FetchDefinedStagesRequest, opts ...grpc.CallOption) (*FetchDefinedStagesResponse, error)
+	// DetermineVersions determines which versions of the artifacts will be used for the given deployment.
+	DetermineVersions(ctx context.Context, in *DetermineVersionsRequest, opts ...grpc.CallOption) (*DetermineVersionsResponse, error)
+	// DetermineStrategy determines which strategy should be used for the given deployment.
+	DetermineStrategy(ctx context.Context, in *DetermineStrategyRequest, opts ...grpc.CallOption) (*DetermineStrategyResponse, error)
+	// BuildStages builds the deployment pipeline stages.
+	BuildStages(ctx context.Context, in *BuildStagesRequest, opts ...grpc.CallOption) (*BuildStagesResponse, error)
+}
+
+type deploymentServiceClient struct {
+	cc grpc.ClientConnInterface
+}
+
+func NewDeploymentServiceClient(cc grpc.ClientConnInterface) DeploymentServiceClient {
+	return &deploymentServiceClient{cc}
+}
+
+func (c *deploymentServiceClient) FetchDefinedStages(ctx context.Context, in *FetchDefinedStagesRequest, opts ...grpc.CallOption) (*FetchDefinedStagesResponse, error) {
+	out := new(FetchDefinedStagesResponse)
+	err := c.cc.Invoke(ctx, "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/FetchDefinedStages", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *deploymentServiceClient) DetermineVersions(ctx context.Context, in *DetermineVersionsRequest, opts ...grpc.CallOption) (*DetermineVersionsResponse, error) {
+	out := new(DetermineVersionsResponse)
+	err := c.cc.Invoke(ctx, "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/DetermineVersions", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *deploymentServiceClient) DetermineStrategy(ctx context.Context, in *DetermineStrategyRequest, opts ...grpc.CallOption) (*DetermineStrategyResponse, error) {
+	out := new(DetermineStrategyResponse)
+	err := c.cc.Invoke(ctx, "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/DetermineStrategy", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *deploymentServiceClient) BuildStages(ctx context.Context, in *BuildStagesRequest, opts ...grpc.CallOption) (*BuildStagesResponse, error) {
+	out := new(BuildStagesResponse)
+	err := c.cc.Invoke(ctx, "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/BuildStages", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// DeploymentServiceServer is the server API for DeploymentService service.
+// All implementations must embed UnimplementedDeploymentServiceServer
+// for forward compatibility
+type DeploymentServiceServer interface {
+	// FetchDefinedStages fetches the defined stages' name which are supported by the plugin.
+	FetchDefinedStages(context.Context, *FetchDefinedStagesRequest) (*FetchDefinedStagesResponse, error)
+	// DetermineVersions determines which versions of the artifacts will be used for the given deployment.
+	DetermineVersions(context.Context, *DetermineVersionsRequest) (*DetermineVersionsResponse, error)
+	// DetermineStrategy determines which strategy should be used for the given deployment.
+	DetermineStrategy(context.Context, *DetermineStrategyRequest) (*DetermineStrategyResponse, error)
+	// BuildStages builds the deployment pipeline stages.
+	BuildStages(context.Context, *BuildStagesRequest) (*BuildStagesResponse, error)
+	mustEmbedUnimplementedDeploymentServiceServer()
+}
+
+// UnimplementedDeploymentServiceServer must be embedded to have forward compatible implementations.
+type UnimplementedDeploymentServiceServer struct {
+}
+
+func (UnimplementedDeploymentServiceServer) FetchDefinedStages(context.Context, *FetchDefinedStagesRequest) (*FetchDefinedStagesResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method FetchDefinedStages not implemented")
+}
+func (UnimplementedDeploymentServiceServer) DetermineVersions(context.Context, *DetermineVersionsRequest) (*DetermineVersionsResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DetermineVersions not implemented")
+}
+func (UnimplementedDeploymentServiceServer) DetermineStrategy(context.Context, *DetermineStrategyRequest) (*DetermineStrategyResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DetermineStrategy not implemented")
+}
+func (UnimplementedDeploymentServiceServer) BuildStages(context.Context, *BuildStagesRequest) (*BuildStagesResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method BuildStages not implemented")
+}
+func (UnimplementedDeploymentServiceServer) mustEmbedUnimplementedDeploymentServiceServer() {}
+
+// UnsafeDeploymentServiceServer may be embedded to opt out of forward compatibility for this service.
+// Use of this interface is not recommended, as added methods to DeploymentServiceServer will
+// result in compilation errors.
+type UnsafeDeploymentServiceServer interface {
+	mustEmbedUnimplementedDeploymentServiceServer()
+}
+
+func RegisterDeploymentServiceServer(s grpc.ServiceRegistrar, srv DeploymentServiceServer) {
+	s.RegisterService(&DeploymentService_ServiceDesc, srv)
+}
+
+func _DeploymentService_FetchDefinedStages_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(FetchDefinedStagesRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(DeploymentServiceServer).FetchDefinedStages(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/FetchDefinedStages",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(DeploymentServiceServer).FetchDefinedStages(ctx, req.(*FetchDefinedStagesRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _DeploymentService_DetermineVersions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DetermineVersionsRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(DeploymentServiceServer).DetermineVersions(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/DetermineVersions",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(DeploymentServiceServer).DetermineVersions(ctx, req.(*DetermineVersionsRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _DeploymentService_DetermineStrategy_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DetermineStrategyRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(DeploymentServiceServer).DetermineStrategy(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/DetermineStrategy",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(DeploymentServiceServer).DetermineStrategy(ctx, req.(*DetermineStrategyRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _DeploymentService_BuildStages_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(BuildStagesRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(DeploymentServiceServer).BuildStages(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/grpc.plugin.deploymentapi.v1alpha1.DeploymentService/BuildStages",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(DeploymentServiceServer).BuildStages(ctx, req.(*BuildStagesRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+// DeploymentService_ServiceDesc is the grpc.ServiceDesc for DeploymentService service.
+// It's only intended for direct use with grpc.RegisterService,
+// and not to be introspected or modified (even as a copy)
+var DeploymentService_ServiceDesc = grpc.ServiceDesc{
+	ServiceName: "grpc.plugin.deploymentapi.v1alpha1.DeploymentService",
+	HandlerType: (*DeploymentServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "FetchDefinedStages",
+			Handler:    _DeploymentService_FetchDefinedStages_Handler,
+		},
+		{
+			MethodName: "DetermineVersions",
+			Handler:    _DeploymentService_DetermineVersions_Handler,
+		},
+		{
+			MethodName: "DetermineStrategy",
+			Handler:    _DeploymentService_DetermineStrategy_Handler,
+		},
+		{
+			MethodName: "BuildStages",
+			Handler:    _DeploymentService_BuildStages_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "pkg/plugin/api/v1alpha1/deployment/api.proto",
+}
diff --git a/pkg/plugin/api/v1alpha1/platform/api.pb.go b/pkg/plugin/api/v1alpha1/platform/api.pb.go
deleted file mode 100644
index 6084138e08..0000000000
--- a/pkg/plugin/api/v1alpha1/platform/api.pb.go
+++ /dev/null
@@ -1,1009 +0,0 @@
-// Copyright 2024 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.
-
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// versions:
-// 	protoc-gen-go v1.27.1
-// 	protoc        v3.21.12
-// source: pkg/plugin/api/v1alpha1/platform/api.proto
-
-package platform
-
-import (
-	_ "github.com/envoyproxy/protoc-gen-validate/validate"
-	model "github.com/pipe-cd/pipecd/pkg/model"
-	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
-	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
-	reflect "reflect"
-	sync "sync"
-)
-
-const (
-	// Verify that this generated code is sufficiently up-to-date.
-	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
-	// Verify that runtime/protoimpl is sufficiently up-to-date.
-	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
-)
-
-type DetermineVersionsRequest struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	Input *PlanPluginInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"`
-}
-
-func (x *DetermineVersionsRequest) Reset() {
-	*x = DetermineVersionsRequest{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[0]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *DetermineVersionsRequest) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*DetermineVersionsRequest) ProtoMessage() {}
-
-func (x *DetermineVersionsRequest) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[0]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use DetermineVersionsRequest.ProtoReflect.Descriptor instead.
-func (*DetermineVersionsRequest) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{0}
-}
-
-func (x *DetermineVersionsRequest) GetInput() *PlanPluginInput {
-	if x != nil {
-		return x.Input
-	}
-	return nil
-}
-
-type DetermineVersionsResponse struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	// The determined versions of the artifacts.
-	Versions []*model.ArtifactVersion `protobuf:"bytes,1,rep,name=versions,proto3" json:"versions,omitempty"`
-}
-
-func (x *DetermineVersionsResponse) Reset() {
-	*x = DetermineVersionsResponse{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[1]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *DetermineVersionsResponse) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*DetermineVersionsResponse) ProtoMessage() {}
-
-func (x *DetermineVersionsResponse) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[1]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use DetermineVersionsResponse.ProtoReflect.Descriptor instead.
-func (*DetermineVersionsResponse) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{1}
-}
-
-func (x *DetermineVersionsResponse) GetVersions() []*model.ArtifactVersion {
-	if x != nil {
-		return x.Versions
-	}
-	return nil
-}
-
-type DetermineStrategyRequest struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	Input *PlanPluginInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"`
-}
-
-func (x *DetermineStrategyRequest) Reset() {
-	*x = DetermineStrategyRequest{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[2]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *DetermineStrategyRequest) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*DetermineStrategyRequest) ProtoMessage() {}
-
-func (x *DetermineStrategyRequest) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[2]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use DetermineStrategyRequest.ProtoReflect.Descriptor instead.
-func (*DetermineStrategyRequest) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{2}
-}
-
-func (x *DetermineStrategyRequest) GetInput() *PlanPluginInput {
-	if x != nil {
-		return x.Input
-	}
-	return nil
-}
-
-type DetermineStrategyResponse struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	// The determined sync strategy.
-	SyncStrategy model.SyncStrategy `protobuf:"varint,1,opt,name=sync_strategy,json=syncStrategy,proto3,enum=model.SyncStrategy" json:"sync_strategy,omitempty"`
-	// Text summary of the determined strategy.
-	Summary string `protobuf:"bytes,2,opt,name=summary,proto3" json:"summary,omitempty"`
-}
-
-func (x *DetermineStrategyResponse) Reset() {
-	*x = DetermineStrategyResponse{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[3]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *DetermineStrategyResponse) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*DetermineStrategyResponse) ProtoMessage() {}
-
-func (x *DetermineStrategyResponse) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[3]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use DetermineStrategyResponse.ProtoReflect.Descriptor instead.
-func (*DetermineStrategyResponse) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{3}
-}
-
-func (x *DetermineStrategyResponse) GetSyncStrategy() model.SyncStrategy {
-	if x != nil {
-		return x.SyncStrategy
-	}
-	return model.SyncStrategy(0)
-}
-
-func (x *DetermineStrategyResponse) GetSummary() string {
-	if x != nil {
-		return x.Summary
-	}
-	return ""
-}
-
-type QuickSyncPlanRequest struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	Input *PlanPluginInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"`
-}
-
-func (x *QuickSyncPlanRequest) Reset() {
-	*x = QuickSyncPlanRequest{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[4]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *QuickSyncPlanRequest) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*QuickSyncPlanRequest) ProtoMessage() {}
-
-func (x *QuickSyncPlanRequest) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[4]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use QuickSyncPlanRequest.ProtoReflect.Descriptor instead.
-func (*QuickSyncPlanRequest) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{4}
-}
-
-func (x *QuickSyncPlanRequest) GetInput() *PlanPluginInput {
-	if x != nil {
-		return x.Input
-	}
-	return nil
-}
-
-type QuickSyncPlanResponse struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	// Stages of deployment pipeline under quick sync strategy.
-	Stages []*model.PipelineStage `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"`
-}
-
-func (x *QuickSyncPlanResponse) Reset() {
-	*x = QuickSyncPlanResponse{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[5]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *QuickSyncPlanResponse) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*QuickSyncPlanResponse) ProtoMessage() {}
-
-func (x *QuickSyncPlanResponse) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[5]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use QuickSyncPlanResponse.ProtoReflect.Descriptor instead.
-func (*QuickSyncPlanResponse) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{5}
-}
-
-func (x *QuickSyncPlanResponse) GetStages() []*model.PipelineStage {
-	if x != nil {
-		return x.Stages
-	}
-	return nil
-}
-
-type PipelineSyncPlanRequest struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	Input *PlanPluginInput `protobuf:"bytes,1,opt,name=input,proto3" json:"input,omitempty"`
-}
-
-func (x *PipelineSyncPlanRequest) Reset() {
-	*x = PipelineSyncPlanRequest{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[6]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *PipelineSyncPlanRequest) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*PipelineSyncPlanRequest) ProtoMessage() {}
-
-func (x *PipelineSyncPlanRequest) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[6]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use PipelineSyncPlanRequest.ProtoReflect.Descriptor instead.
-func (*PipelineSyncPlanRequest) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{6}
-}
-
-func (x *PipelineSyncPlanRequest) GetInput() *PlanPluginInput {
-	if x != nil {
-		return x.Input
-	}
-	return nil
-}
-
-type PipelineSyncPlanResponse struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	// Stages of deployment pipeline under pipeline sync strategy.
-	Stages []*model.PipelineStage `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"`
-}
-
-func (x *PipelineSyncPlanResponse) Reset() {
-	*x = PipelineSyncPlanResponse{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[7]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *PipelineSyncPlanResponse) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*PipelineSyncPlanResponse) ProtoMessage() {}
-
-func (x *PipelineSyncPlanResponse) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[7]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use PipelineSyncPlanResponse.ProtoReflect.Descriptor instead.
-func (*PipelineSyncPlanResponse) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{7}
-}
-
-func (x *PipelineSyncPlanResponse) GetStages() []*model.PipelineStage {
-	if x != nil {
-		return x.Stages
-	}
-	return nil
-}
-
-type PlanPluginInput struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	// The deployment to build a plan for.
-	Deployment *model.Deployment `protobuf:"bytes,1,opt,name=deployment,proto3" json:"deployment,omitempty"`
-	// The remote URL of the deployment source, where plugin can find the deployments sources (manifests).
-	SourceRemoteUrl string `protobuf:"bytes,2,opt,name=source_remote_url,json=sourceRemoteUrl,proto3" json:"source_remote_url,omitempty"`
-	// Last successful commit hash and config file name.
-	// Use to build deployment source object for last successful deployment.
-	LastSuccessfulCommitHash     string `protobuf:"bytes,3,opt,name=last_successful_commit_hash,json=lastSuccessfulCommitHash,proto3" json:"last_successful_commit_hash,omitempty"`
-	LastSuccessfulConfigFileName string `protobuf:"bytes,4,opt,name=last_successful_config_file_name,json=lastSuccessfulConfigFileName,proto3" json:"last_successful_config_file_name,omitempty"`
-	// The configuration of plugin that handles the deployment.
-	PluginConfig []byte `protobuf:"bytes,5,opt,name=plugin_config,json=pluginConfig,proto3" json:"plugin_config,omitempty"`
-}
-
-func (x *PlanPluginInput) Reset() {
-	*x = PlanPluginInput{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[8]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *PlanPluginInput) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*PlanPluginInput) ProtoMessage() {}
-
-func (x *PlanPluginInput) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[8]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use PlanPluginInput.ProtoReflect.Descriptor instead.
-func (*PlanPluginInput) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{8}
-}
-
-func (x *PlanPluginInput) GetDeployment() *model.Deployment {
-	if x != nil {
-		return x.Deployment
-	}
-	return nil
-}
-
-func (x *PlanPluginInput) GetSourceRemoteUrl() string {
-	if x != nil {
-		return x.SourceRemoteUrl
-	}
-	return ""
-}
-
-func (x *PlanPluginInput) GetLastSuccessfulCommitHash() string {
-	if x != nil {
-		return x.LastSuccessfulCommitHash
-	}
-	return ""
-}
-
-func (x *PlanPluginInput) GetLastSuccessfulConfigFileName() string {
-	if x != nil {
-		return x.LastSuccessfulConfigFileName
-	}
-	return ""
-}
-
-func (x *PlanPluginInput) GetPluginConfig() []byte {
-	if x != nil {
-		return x.PluginConfig
-	}
-	return nil
-}
-
-type ExecuteStageRequest struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	Stage       *model.PipelineStage `protobuf:"bytes,1,opt,name=stage,proto3" json:"stage,omitempty"`
-	StageConfig []byte               `protobuf:"bytes,2,opt,name=stage_config,json=stageConfig,proto3" json:"stage_config,omitempty"`
-	PipedConfig []byte               `protobuf:"bytes,3,opt,name=piped_config,json=pipedConfig,proto3" json:"piped_config,omitempty"`
-	Deployment  *model.Deployment    `protobuf:"bytes,4,opt,name=deployment,proto3" json:"deployment,omitempty"`
-}
-
-func (x *ExecuteStageRequest) Reset() {
-	*x = ExecuteStageRequest{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[9]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *ExecuteStageRequest) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*ExecuteStageRequest) ProtoMessage() {}
-
-func (x *ExecuteStageRequest) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[9]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use ExecuteStageRequest.ProtoReflect.Descriptor instead.
-func (*ExecuteStageRequest) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{9}
-}
-
-func (x *ExecuteStageRequest) GetStage() *model.PipelineStage {
-	if x != nil {
-		return x.Stage
-	}
-	return nil
-}
-
-func (x *ExecuteStageRequest) GetStageConfig() []byte {
-	if x != nil {
-		return x.StageConfig
-	}
-	return nil
-}
-
-func (x *ExecuteStageRequest) GetPipedConfig() []byte {
-	if x != nil {
-		return x.PipedConfig
-	}
-	return nil
-}
-
-func (x *ExecuteStageRequest) GetDeployment() *model.Deployment {
-	if x != nil {
-		return x.Deployment
-	}
-	return nil
-}
-
-type ExecuteStageResponse struct {
-	state         protoimpl.MessageState
-	sizeCache     protoimpl.SizeCache
-	unknownFields protoimpl.UnknownFields
-
-	Status model.StageStatus `protobuf:"varint,1,opt,name=status,proto3,enum=model.StageStatus" json:"status,omitempty"`
-	Log    string            `protobuf:"bytes,2,opt,name=log,proto3" json:"log,omitempty"`
-}
-
-func (x *ExecuteStageResponse) Reset() {
-	*x = ExecuteStageResponse{}
-	if protoimpl.UnsafeEnabled {
-		mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[10]
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		ms.StoreMessageInfo(mi)
-	}
-}
-
-func (x *ExecuteStageResponse) String() string {
-	return protoimpl.X.MessageStringOf(x)
-}
-
-func (*ExecuteStageResponse) ProtoMessage() {}
-
-func (x *ExecuteStageResponse) ProtoReflect() protoreflect.Message {
-	mi := &file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[10]
-	if protoimpl.UnsafeEnabled && x != nil {
-		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
-		if ms.LoadMessageInfo() == nil {
-			ms.StoreMessageInfo(mi)
-		}
-		return ms
-	}
-	return mi.MessageOf(x)
-}
-
-// Deprecated: Use ExecuteStageResponse.ProtoReflect.Descriptor instead.
-func (*ExecuteStageResponse) Descriptor() ([]byte, []int) {
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP(), []int{10}
-}
-
-func (x *ExecuteStageResponse) GetStatus() model.StageStatus {
-	if x != nil {
-		return x.Status
-	}
-	return model.StageStatus(0)
-}
-
-func (x *ExecuteStageResponse) GetLog() string {
-	if x != nil {
-		return x.Log
-	}
-	return ""
-}
-
-var File_pkg_plugin_api_v1alpha1_platform_api_proto protoreflect.FileDescriptor
-
-var file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDesc = []byte{
-	0x0a, 0x2a, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2f, 0x61, 0x70, 0x69,
-	0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f,
-	0x72, 0x6d, 0x2f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x20, 0x67, 0x72,
-	0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f,
-	0x72, 0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x1a, 0x17,
-	0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
-	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x16, 0x70, 0x6b, 0x67, 0x2f, 0x6d, 0x6f, 0x64,
-	0x65, 0x6c, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a,
-	0x1a, 0x70, 0x6b, 0x67, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x64, 0x65, 0x70, 0x6c, 0x6f,
-	0x79, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x6d, 0x0a, 0x18, 0x44,
-	0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73,
-	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,
-	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c,
-	0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69,
-	0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x50, 0x6c,
-	0x75, 0x67, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01,
-	0x02, 0x10, 0x01, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x22, 0x4f, 0x0a, 0x19, 0x44, 0x65,
-	0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52,
-	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x32, 0x0a, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69,
-	0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
-	0x6c, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f,
-	0x6e, 0x52, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6d, 0x0a, 0x18, 0x44,
-	0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79,
-	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,
-	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c,
-	0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69,
-	0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x50, 0x6c,
-	0x75, 0x67, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01,
-	0x02, 0x10, 0x01, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x22, 0x6f, 0x0a, 0x19, 0x44, 0x65,
-	0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52,
-	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0d, 0x73, 0x79, 0x6e, 0x63, 0x5f,
-	0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13,
-	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x72, 0x61, 0x74,
-	0x65, 0x67, 0x79, 0x52, 0x0c, 0x73, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67,
-	0x79, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01,
-	0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x69, 0x0a, 0x14, 0x51,
-	0x75, 0x69, 0x63, 0x6b, 0x53, 0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75,
-	0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01,
-	0x28, 0x0b, 0x32, 0x31, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e,
-	0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61,
-	0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e,
-	0x49, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52,
-	0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x22, 0x45, 0x0a, 0x15, 0x51, 0x75, 0x69, 0x63, 0x6b, 0x53,
-	0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
-	0x2c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32,
-	0x14, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65,
-	0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x22, 0x6c, 0x0a,
-	0x17, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61,
-	0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75,
-	0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70,
-	0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70,
-	0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x50,
-	0x6c, 0x75, 0x67, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a,
-	0x01, 0x02, 0x10, 0x01, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x22, 0x48, 0x0a, 0x18, 0x50,
-	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e, 0x52,
-	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65,
-	0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
-	0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x06, 0x73,
-	0x74, 0x61, 0x67, 0x65, 0x73, 0x22, 0xaf, 0x02, 0x0a, 0x0f, 0x50, 0x6c, 0x61, 0x6e, 0x50, 0x6c,
-	0x75, 0x67, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x3b, 0x0a, 0x0a, 0x64, 0x65, 0x70,
-	0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e,
-	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74,
-	0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x0a, 0x64, 0x65, 0x70, 0x6c,
-	0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x33, 0x0a, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
-	0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28,
-	0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x0f, 0x73, 0x6f, 0x75, 0x72,
-	0x63, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x3d, 0x0a, 0x1b, 0x6c,
-	0x61, 0x73, 0x74, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66, 0x75, 0x6c, 0x5f, 0x63,
-	0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
-	0x52, 0x18, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66, 0x75, 0x6c,
-	0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x46, 0x0a, 0x20, 0x6c, 0x61,
-	0x73, 0x74, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66, 0x75, 0x6c, 0x5f, 0x63, 0x6f,
-	0x6e, 0x66, 0x69, 0x67, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04,
-	0x20, 0x01, 0x28, 0x09, 0x52, 0x1c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73,
-	0x73, 0x66, 0x75, 0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x46, 0x69, 0x6c, 0x65, 0x4e, 0x61,
-	0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x6f, 0x6e,
-	0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x70, 0x6c, 0x75, 0x67, 0x69,
-	0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0xe0, 0x01, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63,
-	0x75, 0x74, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
-	0x34, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14,
-	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53,
-	0x74, 0x61, 0x67, 0x65, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x05,
-	0x73, 0x74, 0x61, 0x67, 0x65, 0x12, 0x2a, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x65, 0x5f, 0x63,
-	0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x07, 0xfa, 0x42, 0x04,
-	0x7a, 0x02, 0x10, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69,
-	0x67, 0x12, 0x2a, 0x0a, 0x0c, 0x70, 0x69, 0x70, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69,
-	0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x7a, 0x02, 0x10, 0x01,
-	0x52, 0x0b, 0x70, 0x69, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3b, 0x0a,
-	0x0a, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,
-	0x0b, 0x32, 0x11, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79,
-	0x6d, 0x65, 0x6e, 0x74, 0x42, 0x08, 0xfa, 0x42, 0x05, 0x8a, 0x01, 0x02, 0x10, 0x01, 0x52, 0x0a,
-	0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x54, 0x0a, 0x14, 0x45, 0x78,
-	0x65, 0x63, 0x75, 0x74, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
-	0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01,
-	0x28, 0x0e, 0x32, 0x12, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x53, 0x74, 0x61, 0x67, 0x65,
-	0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x10,
-	0x0a, 0x03, 0x6c, 0x6f, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6c, 0x6f, 0x67,
-	0x32, 0xc5, 0x04, 0x0a, 0x0e, 0x50, 0x6c, 0x61, 0x6e, 0x6e, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76,
-	0x69, 0x63, 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x11, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e,
-	0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3a, 0x2e, 0x67, 0x72, 0x70, 0x63,
-	0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d,
-	0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x74,
-	0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65,
-	0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75,
-	0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69, 0x2e,
-	0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69,
-	0x6e, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
-	0x73, 0x65, 0x22, 0x00, 0x12, 0x8e, 0x01, 0x0a, 0x11, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69,
-	0x6e, 0x65, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x3a, 0x2e, 0x67, 0x72, 0x70,
-	0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72,
-	0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65,
-	0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52,
-	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c,
-	0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69,
-	0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d,
-	0x69, 0x6e, 0x65, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f,
-	0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x0d, 0x51, 0x75, 0x69, 0x63, 0x6b, 0x53,
-	0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x36, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70,
-	0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70,
-	0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x51, 0x75, 0x69, 0x63, 0x6b,
-	0x53, 0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
-	0x37, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c,
-	0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
-	0x61, 0x31, 0x2e, 0x51, 0x75, 0x69, 0x63, 0x6b, 0x53, 0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e,
-	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8b, 0x01, 0x0a, 0x10, 0x50,
-	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e, 0x12,
-	0x39, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c,
-	0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
-	0x61, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x79, 0x6e, 0x63, 0x50,
-	0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x67, 0x72, 0x70,
-	0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72,
-	0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x69,
-	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x79, 0x6e, 0x63, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65,
-	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0x95, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x65,
-	0x63, 0x75, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a,
-	0x0c, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x12, 0x35, 0x2e,
-	0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74,
-	0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
-	0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71,
-	0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x70, 0x6c, 0x75, 0x67,
-	0x69, 0x6e, 0x2e, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x70, 0x69, 0x2e, 0x76,
-	0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x53,
-	0x74, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01,
-	0x42, 0x3c, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70,
-	0x69, 0x70, 0x65, 0x2d, 0x63, 0x64, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x63, 0x64, 0x2f, 0x70, 0x6b,
-	0x67, 0x2f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x61,
-	0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x62, 0x06,
-	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
-}
-
-var (
-	file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescOnce sync.Once
-	file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescData = file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDesc
-)
-
-func file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescGZIP() []byte {
-	file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescOnce.Do(func() {
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescData)
-	})
-	return file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDescData
-}
-
-var file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes = make([]protoimpl.MessageInfo, 11)
-var file_pkg_plugin_api_v1alpha1_platform_api_proto_goTypes = []interface{}{
-	(*DetermineVersionsRequest)(nil),  // 0: grpc.plugin.platformapi.v1alpha1.DetermineVersionsRequest
-	(*DetermineVersionsResponse)(nil), // 1: grpc.plugin.platformapi.v1alpha1.DetermineVersionsResponse
-	(*DetermineStrategyRequest)(nil),  // 2: grpc.plugin.platformapi.v1alpha1.DetermineStrategyRequest
-	(*DetermineStrategyResponse)(nil), // 3: grpc.plugin.platformapi.v1alpha1.DetermineStrategyResponse
-	(*QuickSyncPlanRequest)(nil),      // 4: grpc.plugin.platformapi.v1alpha1.QuickSyncPlanRequest
-	(*QuickSyncPlanResponse)(nil),     // 5: grpc.plugin.platformapi.v1alpha1.QuickSyncPlanResponse
-	(*PipelineSyncPlanRequest)(nil),   // 6: grpc.plugin.platformapi.v1alpha1.PipelineSyncPlanRequest
-	(*PipelineSyncPlanResponse)(nil),  // 7: grpc.plugin.platformapi.v1alpha1.PipelineSyncPlanResponse
-	(*PlanPluginInput)(nil),           // 8: grpc.plugin.platformapi.v1alpha1.PlanPluginInput
-	(*ExecuteStageRequest)(nil),       // 9: grpc.plugin.platformapi.v1alpha1.ExecuteStageRequest
-	(*ExecuteStageResponse)(nil),      // 10: grpc.plugin.platformapi.v1alpha1.ExecuteStageResponse
-	(*model.ArtifactVersion)(nil),     // 11: model.ArtifactVersion
-	(model.SyncStrategy)(0),           // 12: model.SyncStrategy
-	(*model.PipelineStage)(nil),       // 13: model.PipelineStage
-	(*model.Deployment)(nil),          // 14: model.Deployment
-	(model.StageStatus)(0),            // 15: model.StageStatus
-}
-var file_pkg_plugin_api_v1alpha1_platform_api_proto_depIdxs = []int32{
-	8,  // 0: grpc.plugin.platformapi.v1alpha1.DetermineVersionsRequest.input:type_name -> grpc.plugin.platformapi.v1alpha1.PlanPluginInput
-	11, // 1: grpc.plugin.platformapi.v1alpha1.DetermineVersionsResponse.versions:type_name -> model.ArtifactVersion
-	8,  // 2: grpc.plugin.platformapi.v1alpha1.DetermineStrategyRequest.input:type_name -> grpc.plugin.platformapi.v1alpha1.PlanPluginInput
-	12, // 3: grpc.plugin.platformapi.v1alpha1.DetermineStrategyResponse.sync_strategy:type_name -> model.SyncStrategy
-	8,  // 4: grpc.plugin.platformapi.v1alpha1.QuickSyncPlanRequest.input:type_name -> grpc.plugin.platformapi.v1alpha1.PlanPluginInput
-	13, // 5: grpc.plugin.platformapi.v1alpha1.QuickSyncPlanResponse.stages:type_name -> model.PipelineStage
-	8,  // 6: grpc.plugin.platformapi.v1alpha1.PipelineSyncPlanRequest.input:type_name -> grpc.plugin.platformapi.v1alpha1.PlanPluginInput
-	13, // 7: grpc.plugin.platformapi.v1alpha1.PipelineSyncPlanResponse.stages:type_name -> model.PipelineStage
-	14, // 8: grpc.plugin.platformapi.v1alpha1.PlanPluginInput.deployment:type_name -> model.Deployment
-	13, // 9: grpc.plugin.platformapi.v1alpha1.ExecuteStageRequest.stage:type_name -> model.PipelineStage
-	14, // 10: grpc.plugin.platformapi.v1alpha1.ExecuteStageRequest.deployment:type_name -> model.Deployment
-	15, // 11: grpc.plugin.platformapi.v1alpha1.ExecuteStageResponse.status:type_name -> model.StageStatus
-	0,  // 12: grpc.plugin.platformapi.v1alpha1.PlannerService.DetermineVersions:input_type -> grpc.plugin.platformapi.v1alpha1.DetermineVersionsRequest
-	2,  // 13: grpc.plugin.platformapi.v1alpha1.PlannerService.DetermineStrategy:input_type -> grpc.plugin.platformapi.v1alpha1.DetermineStrategyRequest
-	4,  // 14: grpc.plugin.platformapi.v1alpha1.PlannerService.QuickSyncPlan:input_type -> grpc.plugin.platformapi.v1alpha1.QuickSyncPlanRequest
-	6,  // 15: grpc.plugin.platformapi.v1alpha1.PlannerService.PipelineSyncPlan:input_type -> grpc.plugin.platformapi.v1alpha1.PipelineSyncPlanRequest
-	9,  // 16: grpc.plugin.platformapi.v1alpha1.ExecutorService.ExecuteStage:input_type -> grpc.plugin.platformapi.v1alpha1.ExecuteStageRequest
-	1,  // 17: grpc.plugin.platformapi.v1alpha1.PlannerService.DetermineVersions:output_type -> grpc.plugin.platformapi.v1alpha1.DetermineVersionsResponse
-	3,  // 18: grpc.plugin.platformapi.v1alpha1.PlannerService.DetermineStrategy:output_type -> grpc.plugin.platformapi.v1alpha1.DetermineStrategyResponse
-	5,  // 19: grpc.plugin.platformapi.v1alpha1.PlannerService.QuickSyncPlan:output_type -> grpc.plugin.platformapi.v1alpha1.QuickSyncPlanResponse
-	7,  // 20: grpc.plugin.platformapi.v1alpha1.PlannerService.PipelineSyncPlan:output_type -> grpc.plugin.platformapi.v1alpha1.PipelineSyncPlanResponse
-	10, // 21: grpc.plugin.platformapi.v1alpha1.ExecutorService.ExecuteStage:output_type -> grpc.plugin.platformapi.v1alpha1.ExecuteStageResponse
-	17, // [17:22] is the sub-list for method output_type
-	12, // [12:17] is the sub-list for method input_type
-	12, // [12:12] is the sub-list for extension type_name
-	12, // [12:12] is the sub-list for extension extendee
-	0,  // [0:12] is the sub-list for field type_name
-}
-
-func init() { file_pkg_plugin_api_v1alpha1_platform_api_proto_init() }
-func file_pkg_plugin_api_v1alpha1_platform_api_proto_init() {
-	if File_pkg_plugin_api_v1alpha1_platform_api_proto != nil {
-		return
-	}
-	if !protoimpl.UnsafeEnabled {
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*DetermineVersionsRequest); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*DetermineVersionsResponse); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*DetermineStrategyRequest); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*DetermineStrategyResponse); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*QuickSyncPlanRequest); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*QuickSyncPlanResponse); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*PipelineSyncPlanRequest); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*PipelineSyncPlanResponse); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*PlanPluginInput); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*ExecuteStageRequest); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-		file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
-			switch v := v.(*ExecuteStageResponse); i {
-			case 0:
-				return &v.state
-			case 1:
-				return &v.sizeCache
-			case 2:
-				return &v.unknownFields
-			default:
-				return nil
-			}
-		}
-	}
-	type x struct{}
-	out := protoimpl.TypeBuilder{
-		File: protoimpl.DescBuilder{
-			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
-			RawDescriptor: file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDesc,
-			NumEnums:      0,
-			NumMessages:   11,
-			NumExtensions: 0,
-			NumServices:   2,
-		},
-		GoTypes:           file_pkg_plugin_api_v1alpha1_platform_api_proto_goTypes,
-		DependencyIndexes: file_pkg_plugin_api_v1alpha1_platform_api_proto_depIdxs,
-		MessageInfos:      file_pkg_plugin_api_v1alpha1_platform_api_proto_msgTypes,
-	}.Build()
-	File_pkg_plugin_api_v1alpha1_platform_api_proto = out.File
-	file_pkg_plugin_api_v1alpha1_platform_api_proto_rawDesc = nil
-	file_pkg_plugin_api_v1alpha1_platform_api_proto_goTypes = nil
-	file_pkg_plugin_api_v1alpha1_platform_api_proto_depIdxs = nil
-}
diff --git a/pkg/plugin/api/v1alpha1/platform/api_grpc.pb.go b/pkg/plugin/api/v1alpha1/platform/api_grpc.pb.go
deleted file mode 100644
index ea0d204d60..0000000000
--- a/pkg/plugin/api/v1alpha1/platform/api_grpc.pb.go
+++ /dev/null
@@ -1,336 +0,0 @@
-// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
-// versions:
-// - protoc-gen-go-grpc v1.2.0
-// - protoc             v3.21.12
-// source: pkg/plugin/api/v1alpha1/platform/api.proto
-
-package platform
-
-import (
-	context "context"
-	grpc "google.golang.org/grpc"
-	codes "google.golang.org/grpc/codes"
-	status "google.golang.org/grpc/status"
-)
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the grpc package it is being compiled against.
-// Requires gRPC-Go v1.32.0 or later.
-const _ = grpc.SupportPackageIsVersion7
-
-// PlannerServiceClient is the client API for PlannerService service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
-type PlannerServiceClient interface {
-	// DetermineVersions determines which versions of the artifacts will be used for the given deployment.
-	DetermineVersions(ctx context.Context, in *DetermineVersionsRequest, opts ...grpc.CallOption) (*DetermineVersionsResponse, error)
-	// DetermineStrategy determines which strategy should be used for the given deployment.
-	DetermineStrategy(ctx context.Context, in *DetermineStrategyRequest, opts ...grpc.CallOption) (*DetermineStrategyResponse, error)
-	// QuickSyncPlan builds plan for the given deployment using quick sync strategy.
-	QuickSyncPlan(ctx context.Context, in *QuickSyncPlanRequest, opts ...grpc.CallOption) (*QuickSyncPlanResponse, error)
-	// PipelineSyncPlan builds plan for the given deployment using pipeline sync strategy.
-	PipelineSyncPlan(ctx context.Context, in *PipelineSyncPlanRequest, opts ...grpc.CallOption) (*PipelineSyncPlanResponse, error)
-}
-
-type plannerServiceClient struct {
-	cc grpc.ClientConnInterface
-}
-
-func NewPlannerServiceClient(cc grpc.ClientConnInterface) PlannerServiceClient {
-	return &plannerServiceClient{cc}
-}
-
-func (c *plannerServiceClient) DetermineVersions(ctx context.Context, in *DetermineVersionsRequest, opts ...grpc.CallOption) (*DetermineVersionsResponse, error) {
-	out := new(DetermineVersionsResponse)
-	err := c.cc.Invoke(ctx, "/grpc.plugin.platformapi.v1alpha1.PlannerService/DetermineVersions", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *plannerServiceClient) DetermineStrategy(ctx context.Context, in *DetermineStrategyRequest, opts ...grpc.CallOption) (*DetermineStrategyResponse, error) {
-	out := new(DetermineStrategyResponse)
-	err := c.cc.Invoke(ctx, "/grpc.plugin.platformapi.v1alpha1.PlannerService/DetermineStrategy", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *plannerServiceClient) QuickSyncPlan(ctx context.Context, in *QuickSyncPlanRequest, opts ...grpc.CallOption) (*QuickSyncPlanResponse, error) {
-	out := new(QuickSyncPlanResponse)
-	err := c.cc.Invoke(ctx, "/grpc.plugin.platformapi.v1alpha1.PlannerService/QuickSyncPlan", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *plannerServiceClient) PipelineSyncPlan(ctx context.Context, in *PipelineSyncPlanRequest, opts ...grpc.CallOption) (*PipelineSyncPlanResponse, error) {
-	out := new(PipelineSyncPlanResponse)
-	err := c.cc.Invoke(ctx, "/grpc.plugin.platformapi.v1alpha1.PlannerService/PipelineSyncPlan", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// PlannerServiceServer is the server API for PlannerService service.
-// All implementations must embed UnimplementedPlannerServiceServer
-// for forward compatibility
-type PlannerServiceServer interface {
-	// DetermineVersions determines which versions of the artifacts will be used for the given deployment.
-	DetermineVersions(context.Context, *DetermineVersionsRequest) (*DetermineVersionsResponse, error)
-	// DetermineStrategy determines which strategy should be used for the given deployment.
-	DetermineStrategy(context.Context, *DetermineStrategyRequest) (*DetermineStrategyResponse, error)
-	// QuickSyncPlan builds plan for the given deployment using quick sync strategy.
-	QuickSyncPlan(context.Context, *QuickSyncPlanRequest) (*QuickSyncPlanResponse, error)
-	// PipelineSyncPlan builds plan for the given deployment using pipeline sync strategy.
-	PipelineSyncPlan(context.Context, *PipelineSyncPlanRequest) (*PipelineSyncPlanResponse, error)
-	mustEmbedUnimplementedPlannerServiceServer()
-}
-
-// UnimplementedPlannerServiceServer must be embedded to have forward compatible implementations.
-type UnimplementedPlannerServiceServer struct {
-}
-
-func (UnimplementedPlannerServiceServer) DetermineVersions(context.Context, *DetermineVersionsRequest) (*DetermineVersionsResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DetermineVersions not implemented")
-}
-func (UnimplementedPlannerServiceServer) DetermineStrategy(context.Context, *DetermineStrategyRequest) (*DetermineStrategyResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DetermineStrategy not implemented")
-}
-func (UnimplementedPlannerServiceServer) QuickSyncPlan(context.Context, *QuickSyncPlanRequest) (*QuickSyncPlanResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method QuickSyncPlan not implemented")
-}
-func (UnimplementedPlannerServiceServer) PipelineSyncPlan(context.Context, *PipelineSyncPlanRequest) (*PipelineSyncPlanResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method PipelineSyncPlan not implemented")
-}
-func (UnimplementedPlannerServiceServer) mustEmbedUnimplementedPlannerServiceServer() {}
-
-// UnsafePlannerServiceServer may be embedded to opt out of forward compatibility for this service.
-// Use of this interface is not recommended, as added methods to PlannerServiceServer will
-// result in compilation errors.
-type UnsafePlannerServiceServer interface {
-	mustEmbedUnimplementedPlannerServiceServer()
-}
-
-func RegisterPlannerServiceServer(s grpc.ServiceRegistrar, srv PlannerServiceServer) {
-	s.RegisterService(&PlannerService_ServiceDesc, srv)
-}
-
-func _PlannerService_DetermineVersions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DetermineVersionsRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PlannerServiceServer).DetermineVersions(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/grpc.plugin.platformapi.v1alpha1.PlannerService/DetermineVersions",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PlannerServiceServer).DetermineVersions(ctx, req.(*DetermineVersionsRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _PlannerService_DetermineStrategy_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DetermineStrategyRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PlannerServiceServer).DetermineStrategy(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/grpc.plugin.platformapi.v1alpha1.PlannerService/DetermineStrategy",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PlannerServiceServer).DetermineStrategy(ctx, req.(*DetermineStrategyRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _PlannerService_QuickSyncPlan_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(QuickSyncPlanRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PlannerServiceServer).QuickSyncPlan(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/grpc.plugin.platformapi.v1alpha1.PlannerService/QuickSyncPlan",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PlannerServiceServer).QuickSyncPlan(ctx, req.(*QuickSyncPlanRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _PlannerService_PipelineSyncPlan_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(PipelineSyncPlanRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PlannerServiceServer).PipelineSyncPlan(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/grpc.plugin.platformapi.v1alpha1.PlannerService/PipelineSyncPlan",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PlannerServiceServer).PipelineSyncPlan(ctx, req.(*PipelineSyncPlanRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-// PlannerService_ServiceDesc is the grpc.ServiceDesc for PlannerService service.
-// It's only intended for direct use with grpc.RegisterService,
-// and not to be introspected or modified (even as a copy)
-var PlannerService_ServiceDesc = grpc.ServiceDesc{
-	ServiceName: "grpc.plugin.platformapi.v1alpha1.PlannerService",
-	HandlerType: (*PlannerServiceServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "DetermineVersions",
-			Handler:    _PlannerService_DetermineVersions_Handler,
-		},
-		{
-			MethodName: "DetermineStrategy",
-			Handler:    _PlannerService_DetermineStrategy_Handler,
-		},
-		{
-			MethodName: "QuickSyncPlan",
-			Handler:    _PlannerService_QuickSyncPlan_Handler,
-		},
-		{
-			MethodName: "PipelineSyncPlan",
-			Handler:    _PlannerService_PipelineSyncPlan_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "pkg/plugin/api/v1alpha1/platform/api.proto",
-}
-
-// ExecutorServiceClient is the client API for ExecutorService service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
-type ExecutorServiceClient interface {
-	// Execute executes the given stage of the deployment plan.
-	ExecuteStage(ctx context.Context, in *ExecuteStageRequest, opts ...grpc.CallOption) (ExecutorService_ExecuteStageClient, error)
-}
-
-type executorServiceClient struct {
-	cc grpc.ClientConnInterface
-}
-
-func NewExecutorServiceClient(cc grpc.ClientConnInterface) ExecutorServiceClient {
-	return &executorServiceClient{cc}
-}
-
-func (c *executorServiceClient) ExecuteStage(ctx context.Context, in *ExecuteStageRequest, opts ...grpc.CallOption) (ExecutorService_ExecuteStageClient, error) {
-	stream, err := c.cc.NewStream(ctx, &ExecutorService_ServiceDesc.Streams[0], "/grpc.plugin.platformapi.v1alpha1.ExecutorService/ExecuteStage", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &executorServiceExecuteStageClient{stream}
-	if err := x.ClientStream.SendMsg(in); err != nil {
-		return nil, err
-	}
-	if err := x.ClientStream.CloseSend(); err != nil {
-		return nil, err
-	}
-	return x, nil
-}
-
-type ExecutorService_ExecuteStageClient interface {
-	Recv() (*ExecuteStageResponse, error)
-	grpc.ClientStream
-}
-
-type executorServiceExecuteStageClient struct {
-	grpc.ClientStream
-}
-
-func (x *executorServiceExecuteStageClient) Recv() (*ExecuteStageResponse, error) {
-	m := new(ExecuteStageResponse)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-// ExecutorServiceServer is the server API for ExecutorService service.
-// All implementations must embed UnimplementedExecutorServiceServer
-// for forward compatibility
-type ExecutorServiceServer interface {
-	// Execute executes the given stage of the deployment plan.
-	ExecuteStage(*ExecuteStageRequest, ExecutorService_ExecuteStageServer) error
-	mustEmbedUnimplementedExecutorServiceServer()
-}
-
-// UnimplementedExecutorServiceServer must be embedded to have forward compatible implementations.
-type UnimplementedExecutorServiceServer struct {
-}
-
-func (UnimplementedExecutorServiceServer) ExecuteStage(*ExecuteStageRequest, ExecutorService_ExecuteStageServer) error {
-	return status.Errorf(codes.Unimplemented, "method ExecuteStage not implemented")
-}
-func (UnimplementedExecutorServiceServer) mustEmbedUnimplementedExecutorServiceServer() {}
-
-// UnsafeExecutorServiceServer may be embedded to opt out of forward compatibility for this service.
-// Use of this interface is not recommended, as added methods to ExecutorServiceServer will
-// result in compilation errors.
-type UnsafeExecutorServiceServer interface {
-	mustEmbedUnimplementedExecutorServiceServer()
-}
-
-func RegisterExecutorServiceServer(s grpc.ServiceRegistrar, srv ExecutorServiceServer) {
-	s.RegisterService(&ExecutorService_ServiceDesc, srv)
-}
-
-func _ExecutorService_ExecuteStage_Handler(srv interface{}, stream grpc.ServerStream) error {
-	m := new(ExecuteStageRequest)
-	if err := stream.RecvMsg(m); err != nil {
-		return err
-	}
-	return srv.(ExecutorServiceServer).ExecuteStage(m, &executorServiceExecuteStageServer{stream})
-}
-
-type ExecutorService_ExecuteStageServer interface {
-	Send(*ExecuteStageResponse) error
-	grpc.ServerStream
-}
-
-type executorServiceExecuteStageServer struct {
-	grpc.ServerStream
-}
-
-func (x *executorServiceExecuteStageServer) Send(m *ExecuteStageResponse) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-// ExecutorService_ServiceDesc is the grpc.ServiceDesc for ExecutorService service.
-// It's only intended for direct use with grpc.RegisterService,
-// and not to be introspected or modified (even as a copy)
-var ExecutorService_ServiceDesc = grpc.ServiceDesc{
-	ServiceName: "grpc.plugin.platformapi.v1alpha1.ExecutorService",
-	HandlerType: (*ExecutorServiceServer)(nil),
-	Methods:     []grpc.MethodDesc{},
-	Streams: []grpc.StreamDesc{
-		{
-			StreamName:    "ExecuteStage",
-			Handler:       _ExecutorService_ExecuteStage_Handler,
-			ServerStreams: true,
-		},
-	},
-	Metadata: "pkg/plugin/api/v1alpha1/platform/api.proto",
-}
diff --git a/tool/codegen/codegen.sh b/tool/codegen/codegen.sh
index 714fef3b46..8f30860e33 100755
--- a/tool/codegen/codegen.sh
+++ b/tool/codegen/codegen.sh
@@ -19,7 +19,7 @@ goProtoDirs=(
   "pkg/app/server/service/pipedservice"
   "pkg/app/server/service/webservice"
   "pkg/app/helloworld/service"
-  "pkg/plugin/api/v1alpha1/platform"
+  "pkg/plugin/api/v1alpha1/deployment"
   "pkg/app/pipedv1/cmd/piped/service"
 )
 

From 864190792af57549d78af275cb85116fc5d5564b Mon Sep 17 00:00:00 2001
From: khanhtc1202 <khanhtc1202@gmail.com>
Date: Wed, 24 Jul 2024 21:52:25 +0700
Subject: [PATCH 2/2] Fix import

Signed-off-by: khanhtc1202 <khanhtc1202@gmail.com>
---
 pkg/app/pipedv1/plugin/inputs.go            | 4 ++--
 pkg/app/pipedv1/plugin/kubernetes/server.go | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/pkg/app/pipedv1/plugin/inputs.go b/pkg/app/pipedv1/plugin/inputs.go
index d925a9ca29..3fd1c167fe 100644
--- a/pkg/app/pipedv1/plugin/inputs.go
+++ b/pkg/app/pipedv1/plugin/inputs.go
@@ -19,10 +19,10 @@ import (
 
 	"github.com/pipe-cd/pipecd/pkg/app/pipedv1/deploysource"
 	"github.com/pipe-cd/pipecd/pkg/git"
-	"github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/platform"
+	"github.com/pipe-cd/pipecd/pkg/plugin/api/v1alpha1/deployment"
 )
 
-func GetPlanSourceCloner(input *platform.PlanPluginInput) (deploysource.SourceCloner, error) {
+func GetPlanSourceCloner(input *deployment.PlanPluginInput) (deploysource.SourceCloner, error) {
 	gitPath, err := exec.LookPath("git")
 	if err != nil {
 		return nil, err
diff --git a/pkg/app/pipedv1/plugin/kubernetes/server.go b/pkg/app/pipedv1/plugin/kubernetes/server.go
index bae259c163..ef936ec132 100644
--- a/pkg/app/pipedv1/plugin/kubernetes/server.go
+++ b/pkg/app/pipedv1/plugin/kubernetes/server.go
@@ -18,7 +18,7 @@ import (
 	"context"
 	"time"
 
-	"github.com/pipe-cd/pipecd/pkg/app/pipedv1/plugin/platform/kubernetes/planner"
+	"github.com/pipe-cd/pipecd/pkg/app/pipedv1/plugin/kubernetes/planner"
 	"github.com/pipe-cd/pipecd/pkg/cli"
 	"github.com/pipe-cd/pipecd/pkg/rpc"
 	"github.com/spf13/cobra"