From bac25b5e66675524761515e96f25753fa0dbb83b Mon Sep 17 00:00:00 2001 From: thomasjhuang Date: Tue, 4 Mar 2025 00:54:17 -0800 Subject: [PATCH] Add workflow concurrency controller --- .../concurrency/concurrency_controller.go | 541 ++++++++++++++++++ flyteadmin/concurrency/config.go | 25 + flyteadmin/concurrency/core/controller.go | 52 ++ flyteadmin/concurrency/core/policies.go | 107 ++++ .../concurrency/executor/workflow_executor.go | 97 ++++ .../informer/launch_plan_informer.go | 141 +++++ .../repositories/gorm/concurrency_repo.go | 207 +++++++ .../repositories/interfaces/repository.go | 35 ++ flyteidl/clients/go/assets/admin.swagger.json | 35 +- .../gen/pb-es/flyteidl/admin/execution_pb.ts | 8 + .../gen/pb-es/flyteidl/admin/schedule_pb.ts | 115 ++++ .../gen/pb-es/flyteidl/core/execution_pb.ts | 6 + .../gen/pb-go/flyteidl/admin/execution.pb.go | 76 +-- .../gen/pb-go/flyteidl/admin/schedule.pb.go | 272 +++++++-- .../gen/pb-go/flyteidl/core/execution.pb.go | 208 +++---- .../flyteidl/service/admin.swagger.json | 35 +- flyteidl/gen/pb-js/flyteidl.d.ts | 87 ++- flyteidl/gen/pb-js/flyteidl.js | 207 +++++++ .../pb_python/flyteidl/admin/execution_pb2.py | 20 +- .../flyteidl/admin/execution_pb2.pyi | 6 +- .../pb_python/flyteidl/admin/schedule_pb2.py | 14 +- .../pb_python/flyteidl/admin/schedule_pb2.pyi | 32 +- .../pb_python/flyteidl/core/execution_pb2.py | 50 +- .../pb_python/flyteidl/core/execution_pb2.pyi | 2 + flyteidl/gen/pb_rust/flyteidl.admin.rs | 78 +++ flyteidl/gen/pb_rust/flyteidl.core.rs | 3 + .../protos/flyteidl/admin/execution.proto | 5 +- flyteidl/protos/flyteidl/admin/schedule.proto | 31 + flyteidl/protos/flyteidl/core/execution.proto | 1 + 29 files changed, 2281 insertions(+), 215 deletions(-) create mode 100644 flyteadmin/concurrency/concurrency_controller.go create mode 100644 flyteadmin/concurrency/config.go create mode 100644 flyteadmin/concurrency/core/controller.go create mode 100644 flyteadmin/concurrency/core/policies.go create mode 100644 flyteadmin/concurrency/executor/workflow_executor.go create mode 100644 flyteadmin/concurrency/informer/launch_plan_informer.go create mode 100644 flyteadmin/concurrency/repositories/gorm/concurrency_repo.go create mode 100644 flyteadmin/concurrency/repositories/interfaces/repository.go diff --git a/flyteadmin/concurrency/concurrency_controller.go b/flyteadmin/concurrency/concurrency_controller.go new file mode 100644 index 0000000000..670f7fb712 --- /dev/null +++ b/flyteadmin/concurrency/concurrency_controller.go @@ -0,0 +1,541 @@ +package concurrency + +import ( + "context" + "fmt" + "strings" + "sync" + "time" + + "github.com/benbjohnson/clock" + "github.com/prometheus/client_golang/prometheus" + "golang.org/x/time/rate" + "k8s.io/client-go/util/workqueue" + + concurrencyCore "github.com/flyteorg/flyte/flyteadmin/concurrency/core" + "github.com/flyteorg/flyte/flyteadmin/concurrency/executor" + "github.com/flyteorg/flyte/flyteadmin/concurrency/informer" + "github.com/flyteorg/flyte/flyteadmin/concurrency/repositories/interfaces" + "github.com/flyteorg/flyte/flyteadmin/pkg/repositories/models" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" + idlCore "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyte/flytestdlib/logger" + "github.com/flyteorg/flyte/flytestdlib/promutils" +) + +// ConcurrencyController implements core.ConcurrencyController +type ConcurrencyController struct { + repo interfaces.ConcurrencyRepoInterface + policyEvaluator concurrencyCore.PolicyEvaluator + workflowExecutor *executor.WorkflowExecutor + launchPlanInformer *informer.LaunchPlanInformer + + // In-memory tracking of running executions per launch plan. + // Note: map[NamedEntityIdentifier][] + runningExecutions map[string]map[string]bool + runningExecutionsMutex sync.RWMutex + + // Workqueue for pending executions + workqueue workqueue.RateLimitingInterface + + // Processing settings + processingInterval time.Duration + maxRetries int + workers int + + // Metrics + metrics *concurrencyMetrics + + // For controlled shutdown + stopCh chan struct{} + clock clock.Clock +} + +type concurrencyMetrics struct { + pendingExecutions prometheus.Gauge + processingLatency promutils.StopWatch + processingSuccessCount prometheus.Counter + processingFailureCount prometheus.Counter + executionsQueuedCount prometheus.Counter + executionsAbortedCount prometheus.Counter + executionsAllowedCount prometheus.Counter + executionsRejectedCount prometheus.Counter +} + +// NewConcurrencyController creates a new concurrency controller +func NewConcurrencyController( + repo interfaces.ConcurrencyRepoInterface, + workflowExecutor *executor.WorkflowExecutor, + launchPlanInformer *informer.LaunchPlanInformer, + processingInterval time.Duration, + workers int, + maxRetries int, + scope promutils.Scope, +) *ConcurrencyController { + controllerScope := scope.NewSubScope("controller") + + rateLimiter := workqueue.NewMaxOfRateLimiter( + workqueue.NewItemExponentialFailureRateLimiter(100*time.Millisecond, 1000*time.Second), + &workqueue.BucketRateLimiter{Limiter: rate.NewLimiter(rate.Limit(10), 100)}, + ) + + return &ConcurrencyController{ + repo: repo, + policyEvaluator: &concurrencyCore.DefaultPolicyEvaluator{}, + workflowExecutor: workflowExecutor, + launchPlanInformer: launchPlanInformer, + runningExecutions: make(map[string]map[string]bool), + workqueue: workqueue.NewRateLimitingQueue(rateLimiter), + processingInterval: processingInterval, + maxRetries: maxRetries, + workers: workers, + metrics: &concurrencyMetrics{ + pendingExecutions: controllerScope.MustNewGauge("pending_executions", "Number of pending executions"), + processingLatency: controllerScope.MustNewStopWatch("processing_latency", "Time to process pending executions", time.Millisecond), + processingSuccessCount: controllerScope.MustNewCounter("processing_success_count", "Count of successful processing operations"), + processingFailureCount: controllerScope.MustNewCounter("processing_failure_count", "Count of failed processing operations"), + executionsQueuedCount: controllerScope.MustNewCounter("executions_queued_count", "Count of executions that were queued"), + executionsAbortedCount: controllerScope.MustNewCounter("executions_aborted_count", "Count of executions that were aborted"), + executionsAllowedCount: controllerScope.MustNewCounter("executions_allowed_count", "Count of executions that were allowed to run"), + executionsRejectedCount: controllerScope.MustNewCounter("executions_rejected_count", "Count of executions that were rejected"), + }, + stopCh: make(chan struct{}), + clock: clock.New(), + } +} + +// Initialize initializes the controller +func (c *ConcurrencyController) Initialize(ctx context.Context) error { + logger.Info(ctx, "Initializing concurrency controller") + + // Start the launch plan informer + c.launchPlanInformer.Start(ctx) + + // Load initial state + initialState, err := c.loadInitialState(ctx) + if err != nil { + return err + } + + c.runningExecutionsMutex.Lock() + c.runningExecutions = initialState + c.runningExecutionsMutex.Unlock() + + // Start background workers + for i := 0; i < c.workers; i++ { + go c.runWorker(ctx) + } + + // Start periodic processing of pending executions + go c.periodicallyProcessPendingExecutions(ctx) + + return nil +} + +// loadInitialState loads the initial state of running executions from the database +func (c *ConcurrencyController) loadInitialState(ctx context.Context) (map[string]map[string]bool, error) { + // Get all launch plans with concurrency settings + launchPlans, err := c.repo.GetAllLaunchPlansWithConcurrency(ctx) + if err != nil { + return nil, err + } + + runningExecutions := make(map[string]map[string]bool) + + // For each launch plan, get all running executions + for _, lp := range launchPlans { + launchPlanID := idlCore.Identifier{ + Project: lp.Project, + Domain: lp.Domain, + Name: lp.Name, + } + + executions, err := c.repo.GetRunningExecutionsForLaunchPlan(ctx, launchPlanID) + if err != nil { + return nil, err + } + + lpKey := getLaunchPlanKey(lp.Project, lp.Domain, lp.Name) + executionMap := make(map[string]bool) + + for _, exec := range executions { + executionKey := getExecutionKey(exec.Project, exec.Domain, exec.Name) + executionMap[executionKey] = true + } + + if len(executionMap) > 0 { + runningExecutions[lpKey] = executionMap + } + } + + return runningExecutions, nil +} + +// periodicallyProcessPendingExecutions periodically processes pending executions +func (c *ConcurrencyController) periodicallyProcessPendingExecutions(ctx context.Context) { + ticker := c.clock.Ticker(c.processingInterval) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + if err := c.ProcessPendingExecutions(ctx); err != nil { + logger.Errorf(ctx, "Failed to process pending executions: %v", err) + } + case <-c.stopCh: + return + } + } +} + +// ProcessPendingExecutions processes all pending executions +func (c *ConcurrencyController) ProcessPendingExecutions(ctx context.Context) error { + startTime := c.clock.Now() + defer func() { + c.metrics.processingLatency.Observe(startTime, c.clock.Now()) + }() + + pendingExecutions, err := c.repo.GetPendingExecutions(ctx) + if err != nil { + c.metrics.processingFailureCount.Inc() + return err + } + + c.metrics.pendingExecutions.Set(float64(len(pendingExecutions))) + + // Process each pending execution + for _, execution := range pendingExecutions { + // Enqueue the execution for processing + c.workqueue.Add(getExecutionKey(execution.Project, execution.Domain, execution.Name)) + } + + c.metrics.processingSuccessCount.Inc() + return nil +} + +// runWorker is a long-running function that processes items from the workqueue +func (c *ConcurrencyController) runWorker(ctx context.Context) { + for c.processNextWorkItem(ctx) { + } +} + +// processNextWorkItem processes the next item from the workqueue +func (c *ConcurrencyController) processNextWorkItem(ctx context.Context) bool { + item, shutdown := c.workqueue.Get() + if shutdown { + return false + } + + // We call Done here so the workqueue knows we have finished + // processing this item. We also must remember to call Forget if we + // do not want this work item being re-queued. + defer c.workqueue.Done(item) + + executionKey, ok := item.(string) + if !ok { + // As the item in the workqueue is actually invalid, we call + // Forget to ensure it's never queued again. + c.workqueue.Forget(item) + logger.Errorf(ctx, "Expected string in workqueue but got %#v", item) + return true + } + + // Process the execution + err := c.processExecution(ctx, executionKey) + if err == nil { + // If no error occurred, Forget this item so it doesn't get queued again + c.workqueue.Forget(item) + return true + } + + // If an error occurred, requeue unless the maximum number of retries has been reached + if c.workqueue.NumRequeues(item) < c.maxRetries { + logger.Warningf(ctx, "Error processing execution %s: %v", executionKey, err) + c.workqueue.AddRateLimited(item) + return true + } + + // Too many retries + logger.Errorf(ctx, "Dropping execution %s from the workqueue: %v", executionKey, err) + c.workqueue.Forget(item) + return true +} + +// processExecution processes a single execution based on concurrency policy +func (c *ConcurrencyController) processExecution(ctx context.Context, executionKey string) error { + // Parse the execution key + parts := splitExecutionKey(executionKey) + if len(parts) != 3 { + return fmt.Errorf("invalid execution key: %s", executionKey) + } + + // Reconstruct the execution ID + executionID := idlCore.WorkflowExecutionIdentifier{ + Project: parts[0], + Domain: parts[1], + Name: parts[2], + } + + // Get the execution from the database + execution, err := c.repo.GetExecutionByID(ctx, executionID) + if err != nil { + return err + } + + // If execution is no longer pending, skip processing + if execution.Phase != idlCore.WorkflowExecution_PENDING.String() { + return nil + } + + // Get the launch plan ID + launchPlanID := execution.LaunchPlanID + + // Get the concurrency policy for the launch plan + policy, err := c.launchPlanInformer.GetPolicy(ctx, launchPlanID) + if err != nil { + return err + } + + // If no policy, or policy does not have a max concurrency, allow execution + if policy == nil || policy.Max == 0 { + return c.allowExecution(ctx, execution) + } + + // Get current running executions count + runningCount, err := c.GetExecutionCounts(ctx, launchPlanID) + if err != nil { + return err + } + + // Check if the execution can proceed based on concurrency policy + canExecute, reason, err := c.policyEvaluator.CanExecute(ctx, *execution, policy, runningCount) + if err != nil { + return err + } + + if canExecute { + // Handle REPLACE policy - abort oldest execution first if needed + if policy.Policy == admin.ConcurrencyPolicy_REPLACE && runningCount >= int(policy.Max) { + oldestExec, err := c.repo.GetOldestRunningExecution(ctx, launchPlanID) + if err != nil { + return err + } + + if oldestExec != nil { + oldestExecID := idlCore.WorkflowExecutionIdentifier{ + Project: oldestExec.Project, + Domain: oldestExec.Domain, + Name: oldestExec.Name, + } + + err = c.workflowExecutor.AbortExecution(ctx, oldestExecID, "Aborted due to concurrency policy REPLACE") + if err != nil { + return err + } + + c.metrics.executionsAbortedCount.Inc() + + // Untrack the aborted execution + c.UntrackExecution(ctx, oldestExecID) + } + } + + // Allow the execution to proceed + return c.allowExecution(ctx, execution) + } else { + // Handle based on policy + switch policy.Policy { + case admin.ConcurrencyPolicy_ABORT: + // Update execution phase to aborted + err = c.repo.UpdateExecutionPhase(ctx, executionID, idlCore.WorkflowExecution_ABORTED) + if err != nil { + return err + } + + // TODO Create Execution Event on Abort + + c.metrics.executionsAbortedCount.Inc() + return err + + case admin.ConcurrencyPolicy_WAIT: + // Keep the execution in pending state, it will be retried later + c.metrics.executionsQueuedCount.Inc() + return nil + + default: + return fmt.Errorf("unknown concurrency policy: %v", policy.Policy) + } + } +} + +// allowExecution allows an execution to proceed +func (c *ConcurrencyController) allowExecution(ctx context.Context, execution *models.Execution) error { + // Create the execution + err := c.workflowExecutor.CreateExecution(ctx, *execution) + if err != nil { + return err + } + + err = c.TrackExecution(ctx, *execution, execution.LaunchPlanID) + if err != nil { + return err + } + + c.metrics.executionsAllowedCount.Inc() + return nil +} + +// CheckConcurrencyConstraints checks if an execution can proceed +func (c *ConcurrencyController) CheckConcurrencyConstraints(ctx context.Context, launchPlanID idlCore.Identifier) (bool, error) { + policy, err := c.GetConcurrencyPolicy(ctx, launchPlanID) + if err != nil { + return false, err + } + + // If no policy or no max concurrency, allow execution + if policy == nil || policy.Max == 0 { + return true, nil + } + + // Get current running executions count + runningCount, err := c.GetExecutionCounts(ctx, launchPlanID) + if err != nil { + return false, err + } + + // Allow if count is less than max + return runningCount < int(policy.Max), nil +} + +// GetConcurrencyPolicy gets the concurrency policy for a launch plan +func (c *ConcurrencyController) GetConcurrencyPolicy(ctx context.Context, launchPlanID idlCore.Identifier) (*admin.SchedulerPolicy, error) { + return c.launchPlanInformer.GetPolicy(ctx, launchPlanID) +} + +// TrackExecution adds a running execution to tracking +func (c *ConcurrencyController) TrackExecution(ctx context.Context, execution models.Execution, launchPlanID idlCore.Identifier) error { + lpKey := getLaunchPlanKey(launchPlanID.Project, launchPlanID.Domain, launchPlanID.Name) + execKey := getExecutionKey(execution.Project, execution.Domain, execution.Name) + + c.runningExecutionsMutex.Lock() + defer c.runningExecutionsMutex.Unlock() + + execMap, exists := c.runningExecutions[lpKey] + if !exists { + execMap = make(map[string]bool) + c.runningExecutions[lpKey] = execMap + } + + execMap[execKey] = true + return nil +} + +// UntrackExecution removes a completed execution from tracking +func (c *ConcurrencyController) UntrackExecution(ctx context.Context, executionID idlCore.WorkflowExecutionIdentifier) error { + // Get the execution to determine its launch plan + execution, err := c.repo.GetExecutionByID(ctx, executionID) + if err != nil { + return err + } + + if execution == nil { + return nil // Execution already removed + } + + // Convert to launch plan key using the correct launch plan information + lpKey := getLaunchPlanKey(executionID.Project, executionID.Domain, executionID.Name) + execKey := getExecutionKey(executionID.Project, executionID.Domain, executionID.Name) + + c.runningExecutionsMutex.Lock() + defer c.runningExecutionsMutex.Unlock() + + execMap, exists := c.runningExecutions[lpKey] + if !exists { + return nil + } + + delete(execMap, execKey) + + // If no more executions for this launch plan, remove the map + if len(execMap) == 0 { + delete(c.runningExecutions, lpKey) + } + + return nil +} + +// GetExecutionCounts returns the count of running executions for a launch plan +func (c *ConcurrencyController) GetExecutionCounts(ctx context.Context, launchPlanID idlCore.Identifier) (int, error) { + lpKey := getLaunchPlanKey(launchPlanID.Project, launchPlanID.Domain, launchPlanID.Name) + + c.runningExecutionsMutex.RLock() + defer c.runningExecutionsMutex.RUnlock() + + executionMap, exists := c.runningExecutions[lpKey] + if !exists { + return 0, nil + } + + return len(executionMap), nil +} + +// Close gracefully shuts down the controller +func (c *ConcurrencyController) Close() error { + logger.Info(context.Background(), "Shutting down concurrency controller") + close(c.stopCh) + c.workqueue.ShutDown() + return nil +} + +// RecordPendingExecutions implements core.ConcurrencyMetrics +func (c *ConcurrencyController) RecordPendingExecutions(count int64) { + c.metrics.pendingExecutions.Set(float64(count)) +} + +// RecordProcessedExecution implements core.ConcurrencyMetrics +func (c *ConcurrencyController) RecordProcessedExecution(policy string, allowed bool) { + if allowed { + c.metrics.executionsAllowedCount.Inc() + } else { + c.metrics.executionsRejectedCount.Inc() + } +} + +// RecordLatency implements core.ConcurrencyMetrics +func (c *ConcurrencyController) RecordLatency(operation string, startTime time.Time) { + switch operation { + case "processing": + c.metrics.processingLatency.Observe(startTime) + } +} + +// Helper functions + +// getLaunchPlanKey generates a unique string key for a launch plan +func getLaunchPlanKey(project, domain, name string) string { + return fmt.Sprintf("lp:%s:%s:%s", project, domain, name) +} + +// getNamedEntityKey converts a NamedEntityIdentifier to a string key +func getNamedEntityKey(nei admin.NamedEntityIdentifier) string { + return getLaunchPlanKey(nei.Project, nei.Domain, nei.Name) +} + +// getExecutionKey generates a unique string key for an execution +func getExecutionKey(project, domain, name string) string { + return fmt.Sprintf("exec:%s:%s:%s", project, domain, name) +} + +// parseNamedEntityKey converts a string key back to components +func parseNamedEntityKey(key string) (string, string, string, error) { + parts := strings.Split(key, ":") + if len(parts) != 4 || parts[0] != "lp" { + return "", "", "", fmt.Errorf("invalid launch plan key format: %s", key) + } + return parts[1], parts[2], parts[3], nil +} + +// splitExecutionKey splits an execution key into its components +func splitExecutionKey(key string) []string { + return strings.Split(key, ":") +} diff --git a/flyteadmin/concurrency/config.go b/flyteadmin/concurrency/config.go new file mode 100644 index 0000000000..8828badea6 --- /dev/null +++ b/flyteadmin/concurrency/config.go @@ -0,0 +1,25 @@ +package concurrency + +import ( + "github.com/flyteorg/flyte/flytestdlib/config" +) + +const configSectionKey = "concurrency" + +// Config contains settings for the concurrency controller +type Config struct { + // Enable determines whether to enable the concurrency controller + Enable bool `json:"enable" pflag:",Enable the execution concurrency controller"` + + // ProcessingInterval defines how often to check for pending executions + ProcessingInterval config.Duration `json:"processingInterval" pflag:",How often to process pending executions"` + + // Workers defines the number of worker goroutines to process executions + Workers int `json:"workers" pflag:",Number of worker goroutines for processing executions"` + + // MaxRetries defines the maximum number of retries for processing an execution + MaxRetries int `json:"maxRetries" pflag:",Maximum number of retries for processing an execution"` + + // LaunchPlanRefreshInterval defines how often to refresh launch plan concurrency information + LaunchPlanRefreshInterval config.Duration `json:"launchPlanRefreshInterval" pflag:",How often to refresh launch plan concurrency information"` +} diff --git a/flyteadmin/concurrency/core/controller.go b/flyteadmin/concurrency/core/controller.go new file mode 100644 index 0000000000..e217b42dce --- /dev/null +++ b/flyteadmin/concurrency/core/controller.go @@ -0,0 +1,52 @@ +package core + +import ( + "context" + "time" + + "github.com/flyteorg/flyte/flyteadmin/pkg/repositories/models" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" +) + +// ConcurrencyController is the main interface for controlling execution concurrency +// It manages the lifecycle of executions based on concurrency policies +type ConcurrencyController interface { + // Initialize initializes the controller from a previous state if available + Initialize(ctx context.Context) error + + // ProcessPendingExecutions processes all pending executions according to concurrency policies + // It runs in a background process and is periodically scheduled + ProcessPendingExecutions(ctx context.Context) error + + // CheckConcurrencyConstraints checks if an execution can run based on concurrency constraints + // Returns true if the execution can proceed, false otherwise + CheckConcurrencyConstraints(ctx context.Context, launchPlanID core.Identifier) (bool, error) + + // GetConcurrencyPolicy returns the concurrency policy for a given launch plan + GetConcurrencyPolicy(ctx context.Context, launchPlanID core.Identifier) (*admin.SchedulerPolicy, error) + + // TrackExecution adds a running execution to the concurrency tracking + TrackExecution(ctx context.Context, execution models.Execution, launchPlanID core.Identifier) error + + // UntrackExecution removes a completed execution from concurrency tracking + UntrackExecution(ctx context.Context, executionID core.WorkflowExecutionIdentifier) error + + // GetExecutionCounts returns the current count of executions for a launch plan + GetExecutionCounts(ctx context.Context, launchPlanID core.Identifier) (int, error) + + // Close gracefully shuts down the controller + Close() error +} + +// ConcurrencyMetrics defines metrics for the concurrency controller +type ConcurrencyMetrics interface { + // RecordPendingExecutions records the number of pending executions + RecordPendingExecutions(count int64) + + // RecordProcessedExecution records that an execution was processed + RecordProcessedExecution(policy string, allowed bool) + + // RecordLatency records the latency of processing a batch of executions + RecordLatency(operation string, startTime time.Time) +} diff --git a/flyteadmin/concurrency/core/policies.go b/flyteadmin/concurrency/core/policies.go new file mode 100644 index 0000000000..e2904df29d --- /dev/null +++ b/flyteadmin/concurrency/core/policies.go @@ -0,0 +1,107 @@ +package core + +import ( + "context" + + "github.com/flyteorg/flyte/flyteadmin/pkg/repositories/models" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" +) + +// Policy determines what happens when concurrency limit is reached +type Policy int + +const ( + // Wait queues the execution until other executions complete + Wait Policy = iota + + // Abort fails the execution immediately + Abort + + // Replace terminates the oldest running execution + Replace +) + +// FromProto converts a proto policy to a core policy +func PolicyFromProto(policy admin.ConcurrencyPolicy) Policy { + switch policy { + case admin.ConcurrencyPolicy_WAIT: + return Wait + case admin.ConcurrencyPolicy_ABORT: + return Abort + case admin.ConcurrencyPolicy_REPLACE: + return Replace + default: + return Wait // Default to Wait policy + } +} + +// ToString returns a string representation of the policy +func (p Policy) ToString() string { + switch p { + case Wait: + return "WAIT" + case Abort: + return "ABORT" + case Replace: + return "REPLACE" + default: + return "UNKNOWN" + } +} + +// ConcurrencyLevel determines the scope of concurrency limits +type ConcurrencyLevel int + +const ( + // LaunchPlan applies the limit across all versions of a launch plan + LaunchPlan ConcurrencyLevel = iota + + // LaunchPlanVersion applies the limit only to a specific version + LaunchPlanVersion +) + +// FromProto converts a proto level to a core level +func LevelFromProto(level admin.ConcurrencyLevel) ConcurrencyLevel { + switch level { + case admin.ConcurrencyLevel_LAUNCH_PLAN: + return LaunchPlan + case admin.ConcurrencyLevel_LAUNCH_PLAN_VERSION: + return LaunchPlanVersion + default: + return LaunchPlan // Default to launch plan level + } +} + +// PolicyEvaluator evaluates if an execution can proceed based on concurrency policies +type PolicyEvaluator interface { + // CanExecute determines if an execution can proceed based on concurrency constraints + // Returns true if execution can proceed, and an action to take if not + CanExecute(ctx context.Context, execution models.Execution, policy *admin.SchedulerPolicy, runningCount int) (bool, string, error) +} + +// DefaultPolicyEvaluator is the default implementation of PolicyEvaluator +type DefaultPolicyEvaluator struct{} + +// CanExecute implements the PolicyEvaluator interface +func (d *DefaultPolicyEvaluator) CanExecute( + ctx context.Context, + execution models.Execution, + policy *admin.SchedulerPolicy, + runningCount int, +) (bool, string, error) { + if policy == nil || runningCount < int(policy.Max) { + return true, "", nil + } + + corePolicy := PolicyFromProto(policy.Policy) + switch corePolicy { + case Wait: + return false, "Execution queued - waiting for running executions to complete", nil + case Abort: + return false, "Execution aborted - maximum concurrency limit reached", nil + case Replace: + return true, "Execution will replace oldest running execution", nil + default: + return false, "Unknown concurrency policy", nil + } +} diff --git a/flyteadmin/concurrency/executor/workflow_executor.go b/flyteadmin/concurrency/executor/workflow_executor.go new file mode 100644 index 0000000000..3bbe1ec188 --- /dev/null +++ b/flyteadmin/concurrency/executor/workflow_executor.go @@ -0,0 +1,97 @@ +package executor + +import ( + "context" + + "github.com/flyteorg/flyte/flyteadmin/pkg/manager/interfaces" + "github.com/flyteorg/flyte/flyteadmin/pkg/repositories/models" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyte/flytestdlib/logger" + "github.com/flyteorg/flyte/flytestdlib/promutils" + "github.com/prometheus/client_golang/prometheus" +) + +// WorkflowExecutor handles creating and managing workflow executions +type WorkflowExecutor struct { + executionManager interfaces.ExecutionInterface + scope promutils.Scope + metrics *workflowExecutorMetrics +} + +type workflowExecutorMetrics struct { + successfulExecutions prometheus.Counter + failedExecutions prometheus.Counter + abortedExecutions prometheus.Counter +} + +// NewWorkflowExecutor creates a new WorkflowExecutor +func NewWorkflowExecutor( + executionManager interfaces.ExecutionInterface, + scope promutils.Scope, +) *WorkflowExecutor { + executorScope := scope.NewSubScope("workflow_executor") + return &WorkflowExecutor{ + executionManager: executionManager, + scope: executorScope, + metrics: &workflowExecutorMetrics{ + successfulExecutions: executorScope.MustNewCounter("successful_executions", "Count of successful execution creations"), + failedExecutions: executorScope.MustNewCounter("failed_executions", "Count of failed execution creations"), + abortedExecutions: executorScope.MustNewCounter("aborted_executions", "Count of aborted executions"), + }, + } +} + +// CreateExecution creates a workflow execution in the system +func (w *WorkflowExecutor) CreateExecution(ctx context.Context, execution models.Execution) error { + // Ensure we have the proper identifier + launchPlanIdentifier := &core.Identifier{ + ResourceType: core.ResourceType_LAUNCH_PLAN, + Project: execution.Project, + Domain: execution.Domain, + Name: execution.Name, + } + + // Create execution metadata + metadata := &admin.ExecutionMetadata{ + Mode: admin.ExecutionMetadata_SCHEDULED, // Or whatever mode is appropriate + } + + // Create the execution request + executionRequest := &admin.ExecutionCreateRequest{ + Domain: execution.Domain, + Project: execution.Project, + Name: execution.Name, + Spec: &admin.ExecutionSpec{ + LaunchPlan: launchPlanIdentifier, + Metadata: metadata, + }, + } + + // Create the workflow execution + _, err := w.executionManager.CreateExecution(ctx, executionRequest, nil) + if err != nil { + w.metrics.failedExecutions.Inc() + logger.Errorf(ctx, "Failed to create execution: %v", err) + return err + } + + w.metrics.successfulExecutions.Inc() + return nil +} + +// AbortExecution aborts an existing workflow execution +func (w *WorkflowExecutor) AbortExecution(ctx context.Context, executionID core.WorkflowExecutionIdentifier, cause string) error { + _, err := w.executionManager.TerminateExecution(ctx, &admin.ExecutionTerminateRequest{ + Id: &executionID, + Cause: cause, + }) + + if err != nil { + logger.Errorf(ctx, "Failed to abort execution %v: %v", executionID, err) + return err + } + + w.metrics.abortedExecutions.Inc() + return nil +} diff --git a/flyteadmin/concurrency/informer/launch_plan_informer.go b/flyteadmin/concurrency/informer/launch_plan_informer.go new file mode 100644 index 0000000000..6be6a6c409 --- /dev/null +++ b/flyteadmin/concurrency/informer/launch_plan_informer.go @@ -0,0 +1,141 @@ +package informer + +import ( + "context" + "sync" + "time" + + "github.com/flyteorg/flyte/flyteadmin/concurrency/repositories/interfaces" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyte/flytestdlib/logger" + "github.com/flyteorg/flyte/flytestdlib/promutils" + "github.com/prometheus/client_golang/prometheus" +) + +// LaunchPlanInformer caches and periodically refreshes launch plan concurrency information +type LaunchPlanInformer struct { + repo interfaces.ConcurrencyRepoInterface + refreshInterval time.Duration + launchPlans map[string]*admin.SchedulerPolicy + mutex sync.RWMutex + scope promutils.Scope + metrics *launchPlanInformerMetrics +} + +type launchPlanInformerMetrics struct { + refreshSuccessCount prometheus.Counter + refreshFailureCount prometheus.Counter + refreshDuration promutils.StopWatch + cachedLaunchPlans prometheus.Gauge +} + +// NewLaunchPlanInformer creates a new LaunchPlanInformer +func NewLaunchPlanInformer( + repo interfaces.ConcurrencyRepoInterface, + refreshInterval time.Duration, + scope promutils.Scope, +) *LaunchPlanInformer { + informerScope := scope.NewSubScope("launch_plan_informer") + return &LaunchPlanInformer{ + repo: repo, + refreshInterval: refreshInterval, + launchPlans: make(map[string]*admin.SchedulerPolicy), + scope: informerScope, + metrics: &launchPlanInformerMetrics{ + refreshSuccessCount: informerScope.MustNewCounter("refresh_success_count", "Count of successful refreshes"), + refreshFailureCount: informerScope.MustNewCounter("refresh_failure_count", "Count of failed refreshes"), + refreshDuration: informerScope.MustNewStopWatch("refresh_duration", "Duration of refresh operations", time.Millisecond), + cachedLaunchPlans: informerScope.MustNewGauge("cached_launch_plans", "Number of cached launch plans"), + }, + } +} + +// Start begins the periodic refresh of launch plan information +func (l *LaunchPlanInformer) Start(ctx context.Context) { + ticker := time.NewTicker(l.refreshInterval) + go func() { + // Perform an initial refresh + err := l.Refresh(ctx) + if err != nil { + logger.Errorf(ctx, "Initial launch plan informer refresh failed: %v", err) + } + + for { + select { + case <-ctx.Done(): + ticker.Stop() + return + case <-ticker.C: + err := l.Refresh(ctx) + if err != nil { + logger.Errorf(ctx, "Launch plan informer refresh failed: %v", err) + } + } + } + }() +} + +// Refresh updates the cached launch plan information +func (l *LaunchPlanInformer) Refresh(ctx context.Context) error { + timer := l.metrics.refreshDuration.Start() + defer timer.Stop() + + launchPlans, err := l.repo.GetAllLaunchPlansWithConcurrency(ctx) + if err != nil { + l.metrics.refreshFailureCount.Inc() + return err + } + + newLaunchPlans := make(map[string]*admin.SchedulerPolicy) + for _, lp := range launchPlans { + if lp.SchedulerPolicy != nil { + lpID := getLaunchPlanKey(lp.Project, lp.Domain, lp.Name) + policy := &admin.SchedulerPolicy{ + Max: lp.SchedulerPolicy.MaxConcurrent, + Policy: admin.ConcurrencyPolicy(lp.SchedulerPolicy.Policy), + Level: admin.ConcurrencyLevel(lp.SchedulerPolicy.Level), + } + newLaunchPlans[lpID] = policy + } + } + + l.mutex.Lock() + l.launchPlans = newLaunchPlans + l.mutex.Unlock() + + l.metrics.cachedLaunchPlans.Set(float64(len(newLaunchPlans))) + l.metrics.refreshSuccessCount.Inc() + return nil +} + +// GetPolicy returns the cached concurrency policy for a launch plan +func (l *LaunchPlanInformer) GetPolicy(ctx context.Context, id core.Identifier) (*admin.SchedulerPolicy, error) { + l.mutex.RLock() + defer l.mutex.RUnlock() + + lpKey := getLaunchPlanKey(id.Project, id.Domain, id.Name) + policy, exists := l.launchPlans[lpKey] + if !exists { + // If not in cache, try to fetch directly + lp, err := l.repo.GetActiveLaunchPlanWithConcurrency(ctx, id) + if err != nil { + return nil, err + } + + if lp != nil && lp.SchedulerPolicy != nil { + return &admin.SchedulerPolicy{ + Max: lp.SchedulerPolicy.MaxConcurrent, + Policy: admin.ConcurrencyPolicy(lp.SchedulerPolicy.Policy), + Level: admin.ConcurrencyLevel(lp.SchedulerPolicy.Level), + }, nil + } + return nil, nil + } + return policy, nil +} + +// getLaunchPlanKey generates a unique string key for a launch plan +func getLaunchPlanKey(project, domain, name string) string { + return project + ":" + domain + ":" + name +} diff --git a/flyteadmin/concurrency/repositories/gorm/concurrency_repo.go b/flyteadmin/concurrency/repositories/gorm/concurrency_repo.go new file mode 100644 index 0000000000..209a79d8c5 --- /dev/null +++ b/flyteadmin/concurrency/repositories/gorm/concurrency_repo.go @@ -0,0 +1,207 @@ +package gorm + +import ( + "context" + "fmt" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" + + "gorm.io/gorm" + + "github.com/flyteorg/flyte/flyteadmin/concurrency/repositories/interfaces" + adminRepoInterfaces "github.com/flyteorg/flyte/flyteadmin/pkg/repositories/interfaces" + "github.com/flyteorg/flyte/flyteadmin/pkg/repositories/models" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyte/flytestdlib/promutils" +) + +// GormConcurrencyRepo implements the ConcurrencyRepoInterface using GORM +type GormConcurrencyRepo struct { + db *gorm.DB + scope promutils.Scope + repo adminRepoInterfaces.Repository +} + +// NewGormConcurrencyRepo creates a new GORM-based concurrency repository +func NewGormConcurrencyRepo( + db *gorm.DB, + repo adminRepoInterfaces.Repository, + scope promutils.Scope, +) interfaces.ConcurrencyRepoInterface { + return &GormConcurrencyRepo{ + db: db, + repo: repo, + scope: scope, + } +} + +// GetPendingExecutions retrieves all pending executions ordered by creation time +func (r *GormConcurrencyRepo) GetPendingExecutions(ctx context.Context) ([]models.Execution, error) { + var executions []models.Execution + + // Query pending executions in FIFO order + tx := r.db.Where("phase = ?", core.WorkflowExecution_PENDING.String()). + Order("created_at ASC"). + Find(&executions) + + if tx.Error != nil { + return nil, fmt.Errorf("failed to get pending executions: %w", tx.Error) + } + + return executions, nil +} + +// GetRunningExecutionsForLaunchPlan retrieves all non-terminal executions for a launch plan +func (r *GormConcurrencyRepo) GetRunningExecutionsForLaunchPlan( + ctx context.Context, + launchPlanID core.Identifier, +) ([]models.Execution, error) { + var executions []models.Execution + + // Get non-terminal executions for the given launch plan + tx := r.db.Where("launch_plan_project = ? AND launch_plan_domain = ? AND launch_plan_name = ? AND phase NOT IN (?, ?, ?, ?)", + launchPlanID.Project, + launchPlanID.Domain, + launchPlanID.Name, + core.WorkflowExecution_SUCCEEDED.String(), + core.WorkflowExecution_FAILED.String(), + core.WorkflowExecution_ABORTED.String(), + core.WorkflowExecution_TIMED_OUT.String(), + ).Find(&executions) + + if tx.Error != nil { + return nil, fmt.Errorf("failed to get running executions for launch plan: %w", tx.Error) + } + + return executions, nil +} + +// GetRunningExecutionsCount gets the count of non-terminal executions for a launch plan +func (r *GormConcurrencyRepo) GetRunningExecutionsCount( + ctx context.Context, + launchPlanID core.Identifier, +) (int, error) { + var count int64 + + // Count non-terminal executions for the given launch plan + tx := r.db.Model(&models.Execution{}). + Where("launch_plan_project = ? AND launch_plan_domain = ? AND launch_plan_name = ? AND phase NOT IN (?, ?, ?, ?)", + launchPlanID.Project, + launchPlanID.Domain, + launchPlanID.Name, + core.WorkflowExecution_SUCCEEDED.String(), + core.WorkflowExecution_FAILED.String(), + core.WorkflowExecution_ABORTED.String(), + core.WorkflowExecution_TIMED_OUT.String(), + ).Count(&count) + + if tx.Error != nil { + return 0, fmt.Errorf("failed to count running executions for launch plan: %w", tx.Error) + } + + return int(count), nil +} + +// GetActiveLaunchPlanWithConcurrency gets a launch plan with concurrency settings +func (r *GormConcurrencyRepo) GetActiveLaunchPlanWithConcurrency( + ctx context.Context, + launchPlanID core.Identifier, +) (*models.LaunchPlan, error) { + var launchPlan models.LaunchPlan + + // Get the active launch plan with its concurrency settings + tx := r.db.Where("project = ? AND domain = ? AND name = ? AND state = ? AND scheduler_policy IS NOT NULL", + launchPlanID.Project, + launchPlanID.Domain, + launchPlanID.Name, + admin.LaunchPlanState_ACTIVE.String(), + ).First(&launchPlan) + + if tx.Error != nil { + if tx.Error == gorm.ErrRecordNotFound { + return nil, nil // No active launch plan with concurrency + } + return nil, fmt.Errorf("failed to get active launch plan: %w", tx.Error) + } + + return &launchPlan, nil +} + +// UpdateExecutionPhase updates the phase of an execution +func (r *GormConcurrencyRepo) UpdateExecutionPhase( + ctx context.Context, + executionID core.WorkflowExecutionIdentifier, + phase core.WorkflowExecution_Phase, +) error { + tx := r.db.Model(&models.Execution{}). + Where("project = ? AND domain = ? AND name = ?", + executionID.Project, + executionID.Domain, + executionID.Name, + ). + Update("phase", phase.String()) + + if tx.Error != nil { + return fmt.Errorf("failed to update execution phase: %w", tx.Error) + } + + return nil +} + +// GetOldestRunningExecution gets the oldest running execution for a launch plan +func (r *GormConcurrencyRepo) GetOldestRunningExecution( + ctx context.Context, + launchPlanID core.Identifier, +) (*models.Execution, error) { + var execution models.Execution + + // Get the oldest running execution for the given launch plan + tx := r.db.Where("launch_plan_project = ? AND launch_plan_domain = ? AND launch_plan_name = ? AND phase NOT IN (?, ?, ?, ?)", + launchPlanID.Project, + launchPlanID.Domain, + launchPlanID.Name, + core.WorkflowExecution_SUCCEEDED.String(), + core.WorkflowExecution_FAILED.String(), + core.WorkflowExecution_ABORTED.String(), + core.WorkflowExecution_TIMED_OUT.String(), + ). + Order("created_at ASC"). + First(&execution) + + if tx.Error != nil { + if tx.Error == gorm.ErrRecordNotFound { + return nil, nil // No running executions + } + return nil, fmt.Errorf("failed to get oldest running execution: %w", tx.Error) + } + + return &execution, nil +} + +// GetAllLaunchPlansWithConcurrency gets all launch plans with concurrency settings +func (r *GormConcurrencyRepo) GetAllLaunchPlansWithConcurrency(ctx context.Context) ([]models.LaunchPlan, error) { + var launchPlans []models.LaunchPlan + + // Get all launch plans with concurrency settings + tx := r.db.Where("scheduler_policy IS NOT NULL").Find(&launchPlans) + + if tx.Error != nil { + return nil, fmt.Errorf("failed to get launch plans with concurrency: %w", tx.Error) + } + + return launchPlans, nil +} + +// GetExecutionByID retrieves an execution by its identifier +func (r *GormConcurrencyRepo) GetExecutionByID(ctx context.Context, executionID core.WorkflowExecutionIdentifier) (*models.Execution, error) { + var execution models.Execution + + tx := r.db.Where("project = ? AND domain = ? AND name = ?", + executionID.Project, executionID.Domain, executionID.Name). + First(&execution) + + if tx.Error != nil { + return nil, tx.Error + } + + return &execution, nil +} diff --git a/flyteadmin/concurrency/repositories/interfaces/repository.go b/flyteadmin/concurrency/repositories/interfaces/repository.go new file mode 100644 index 0000000000..d0464f9a4a --- /dev/null +++ b/flyteadmin/concurrency/repositories/interfaces/repository.go @@ -0,0 +1,35 @@ +package interfaces + +import ( + "context" + + "github.com/flyteorg/flyte/flyteadmin/pkg/repositories/models" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" +) + +// ConcurrencyRepoInterface defines the interface for accessing concurrency-related data +type ConcurrencyRepoInterface interface { + // GetPendingExecutions retrieves all pending executions ordered by creation time + GetPendingExecutions(ctx context.Context) ([]models.Execution, error) + + // GetRunningExecutionsForLaunchPlan retrieves all non-terminal executions for a launch plan + GetRunningExecutionsForLaunchPlan(ctx context.Context, launchPlanID core.Identifier) ([]models.Execution, error) + + // GetRunningExecutionsCount gets the count of non-terminal executions for a launch plan + GetRunningExecutionsCount(ctx context.Context, launchPlanID core.Identifier) (int, error) + + // GetActiveLaunchPlanWithConcurrency gets a launch plan with concurrency settings + GetActiveLaunchPlanWithConcurrency(ctx context.Context, launchPlanID core.Identifier) (*models.LaunchPlan, error) + + // UpdateExecutionPhase updates the phase of an execution + UpdateExecutionPhase(ctx context.Context, executionID core.WorkflowExecutionIdentifier, phase core.WorkflowExecution_Phase) error + + // GetOldestRunningExecution gets the oldest running execution for a launch plan + GetOldestRunningExecution(ctx context.Context, launchPlanID core.Identifier) (*models.Execution, error) + + // GetAllLaunchPlansWithConcurrency gets all launch plans with concurrency settings + GetAllLaunchPlansWithConcurrency(ctx context.Context) ([]models.LaunchPlan, error) + + // GetExecutionByID returns an execution when given an executionID + GetExecutionByID(ctx context.Context, executionID core.WorkflowExecutionIdentifier) (*models.Execution, error) +} diff --git a/flyteidl/clients/go/assets/admin.swagger.json b/flyteidl/clients/go/assets/admin.swagger.json index 52fbff78d6..07c00867b7 100644 --- a/flyteidl/clients/go/assets/admin.swagger.json +++ b/flyteidl/clients/go/assets/admin.swagger.json @@ -4580,6 +4580,17 @@ } } }, + "adminConcurrencyPolicy": { + "type": "string", + "enum": [ + "UNSPECIFIED", + "WAIT", + "ABORT", + "REPLACE" + ], + "default": "UNSPECIFIED", + "title": "- WAIT: wait for previous executions to terminate before starting a new one\n - ABORT: fail the CreateExecution request and do not permit the execution to start\n - REPLACE: terminate the oldest execution when the concurrency limit is reached and immediately begin proceeding with the new execution" + }, "adminCronSchedule": { "type": "object", "properties": { @@ -5095,6 +5106,10 @@ "principal": { "type": "string", "title": "Identifies the entity (if any) responsible for causing the state change of the execution" + }, + "description": { + "type": "string", + "title": "Includes the reason for the `PENDING` phase" } } }, @@ -5989,10 +6004,27 @@ "kickoff_time_input_arg": { "type": "string", "description": "Name of the input variable that the kickoff time will be supplied to when the workflow is kicked off." + }, + "scheduler_policy": { + "$ref": "#/definitions/adminSchedulerPolicy" } }, "description": "Defines complete set of information required to trigger an execution on a schedule." }, + "adminSchedulerPolicy": { + "type": "object", + "properties": { + "max": { + "type": "integer", + "format": "int64", + "title": "Defines how many executions with this launch plan can run in parallel" + }, + "policy": { + "$ref": "#/definitions/adminConcurrencyPolicy", + "description": "Defines how to handle the execution when the max concurrency is reached." + } + } + }, "adminSlackNotification": { "type": "object", "properties": { @@ -8611,7 +8643,8 @@ "FAILED", "ABORTED", "TIMED_OUT", - "ABORTING" + "ABORTING", + "PENDING" ], "default": "UNDEFINED" }, diff --git a/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts b/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts index 5ba9f62ca6..9194dbe0cc 100644 --- a/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts +++ b/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts @@ -1447,6 +1447,13 @@ export class ExecutionStateChangeDetails extends Message) { super(); proto3.util.initPartial(data, this); @@ -1458,6 +1465,7 @@ export class ExecutionStateChangeDetails extends Message): ExecutionStateChangeDetails { diff --git a/flyteidl/gen/pb-es/flyteidl/admin/schedule_pb.ts b/flyteidl/gen/pb-es/flyteidl/admin/schedule_pb.ts index b691123586..81366120ee 100644 --- a/flyteidl/gen/pb-es/flyteidl/admin/schedule_pb.ts +++ b/flyteidl/gen/pb-es/flyteidl/admin/schedule_pb.ts @@ -34,6 +34,68 @@ proto3.util.setEnumType(FixedRateUnit, "flyteidl.admin.FixedRateUnit", [ { no: 2, name: "DAY" }, ]); +/** + * @generated from enum flyteidl.admin.ConcurrencyPolicy + */ +export enum ConcurrencyPolicy { + /** + * @generated from enum value: UNSPECIFIED = 0; + */ + UNSPECIFIED = 0, + + /** + * wait for previous executions to terminate before starting a new one + * + * @generated from enum value: WAIT = 1; + */ + WAIT = 1, + + /** + * fail the CreateExecution request and do not permit the execution to start + * + * @generated from enum value: ABORT = 2; + */ + ABORT = 2, + + /** + * terminate the oldest execution when the concurrency limit is reached and immediately begin proceeding with the new execution + * + * @generated from enum value: REPLACE = 3; + */ + REPLACE = 3, +} +// Retrieve enum metadata with: proto3.getEnumType(ConcurrencyPolicy) +proto3.util.setEnumType(ConcurrencyPolicy, "flyteidl.admin.ConcurrencyPolicy", [ + { no: 0, name: "UNSPECIFIED" }, + { no: 1, name: "WAIT" }, + { no: 2, name: "ABORT" }, + { no: 3, name: "REPLACE" }, +]); + +/** + * @generated from enum flyteidl.admin.ConcurrencyLevel + */ +export enum ConcurrencyLevel { + /** + * Applies concurrency limits across all launch plan versions. + * + * @generated from enum value: LAUNCH_PLAN = 0; + */ + LAUNCH_PLAN = 0, + + /** + * Applies concurrency at the versioned launch plan level + * + * @generated from enum value: LAUNCH_PLAN_VERSION = 1; + */ + LAUNCH_PLAN_VERSION = 1, +} +// Retrieve enum metadata with: proto3.getEnumType(ConcurrencyLevel) +proto3.util.setEnumType(ConcurrencyLevel, "flyteidl.admin.ConcurrencyLevel", [ + { no: 0, name: "LAUNCH_PLAN" }, + { no: 1, name: "LAUNCH_PLAN_VERSION" }, +]); + /** * Option for schedules run at a certain frequency e.g. every 2 minutes. * @@ -171,6 +233,11 @@ export class Schedule extends Message { */ kickoffTimeInputArg = ""; + /** + * @generated from field: flyteidl.admin.SchedulerPolicy scheduler_policy = 5; + */ + schedulerPolicy?: SchedulerPolicy; + constructor(data?: PartialMessage) { super(); proto3.util.initPartial(data, this); @@ -183,6 +250,7 @@ export class Schedule extends Message { { no: 2, name: "rate", kind: "message", T: FixedRate, oneof: "ScheduleExpression" }, { no: 4, name: "cron_schedule", kind: "message", T: CronSchedule, oneof: "ScheduleExpression" }, { no: 3, name: "kickoff_time_input_arg", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 5, name: "scheduler_policy", kind: "message", T: SchedulerPolicy }, ]); static fromBinary(bytes: Uint8Array, options?: Partial): Schedule { @@ -202,3 +270,50 @@ export class Schedule extends Message { } } +/** + * @generated from message flyteidl.admin.SchedulerPolicy + */ +export class SchedulerPolicy extends Message { + /** + * Defines how many executions with this launch plan can run in parallel + * + * @generated from field: uint32 max = 1; + */ + max = 0; + + /** + * Defines how to handle the execution when the max concurrency is reached. + * + * @generated from field: flyteidl.admin.ConcurrencyPolicy policy = 2; + */ + policy = ConcurrencyPolicy.UNSPECIFIED; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "flyteidl.admin.SchedulerPolicy"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "max", kind: "scalar", T: 13 /* ScalarType.UINT32 */ }, + { no: 2, name: "policy", kind: "enum", T: proto3.getEnumType(ConcurrencyPolicy) }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): SchedulerPolicy { + return new SchedulerPolicy().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): SchedulerPolicy { + return new SchedulerPolicy().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): SchedulerPolicy { + return new SchedulerPolicy().fromJsonString(jsonString, options); + } + + static equals(a: SchedulerPolicy | PlainMessage | undefined, b: SchedulerPolicy | PlainMessage | undefined): boolean { + return proto3.util.equals(SchedulerPolicy, a, b); + } +} + diff --git a/flyteidl/gen/pb-es/flyteidl/core/execution_pb.ts b/flyteidl/gen/pb-es/flyteidl/core/execution_pb.ts index d9d0a71718..d89df2d8fb 100644 --- a/flyteidl/gen/pb-es/flyteidl/core/execution_pb.ts +++ b/flyteidl/gen/pb-es/flyteidl/core/execution_pb.ts @@ -92,6 +92,11 @@ export enum WorkflowExecution_Phase { * @generated from enum value: ABORTING = 9; */ ABORTING = 9, + + /** + * @generated from enum value: PENDING = 10; + */ + PENDING = 10, } // Retrieve enum metadata with: proto3.getEnumType(WorkflowExecution_Phase) proto3.util.setEnumType(WorkflowExecution_Phase, "flyteidl.core.WorkflowExecution.Phase", [ @@ -105,6 +110,7 @@ proto3.util.setEnumType(WorkflowExecution_Phase, "flyteidl.core.WorkflowExecutio { no: 7, name: "ABORTED" }, { no: 8, name: "TIMED_OUT" }, { no: 9, name: "ABORTING" }, + { no: 10, name: "PENDING" }, ]); /** diff --git a/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go b/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go index d4b5ea85e1..b27c9c496d 100644 --- a/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go @@ -1772,6 +1772,8 @@ type ExecutionStateChangeDetails struct { OccurredAt *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=occurred_at,json=occurredAt,proto3" json:"occurred_at,omitempty"` // Identifies the entity (if any) responsible for causing the state change of the execution Principal string `protobuf:"bytes,3,opt,name=principal,proto3" json:"principal,omitempty"` + // Includes the reason for the `PENDING` phase + Description string `protobuf:"bytes,4,opt,name=description,proto3" json:"description,omitempty"` } func (x *ExecutionStateChangeDetails) Reset() { @@ -1827,6 +1829,13 @@ func (x *ExecutionStateChangeDetails) GetPrincipal() string { return "" } +func (x *ExecutionStateChangeDetails) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + type ExecutionUpdateResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2307,7 +2316,7 @@ var file_flyteidl_admin_execution_proto_rawDesc = []byte{ 0x64, 0x12, 0x34, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xae, 0x01, 0x0a, 0x1b, 0x45, 0x78, 0x65, 0x63, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xd0, 0x01, 0x0a, 0x1b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x12, 0x34, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, @@ -2318,37 +2327,40 @@ var file_flyteidl_admin_execution_proto_rawDesc = []byte{ 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x6f, 0x63, 0x63, 0x75, 0x72, 0x72, 0x65, 0x64, 0x41, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, - 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x22, 0x19, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x76, 0x0a, 0x22, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, - 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x22, 0x4e, 0x0a, 0x23, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x27, 0x0a, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x13, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, - 0x2e, 0x53, 0x70, 0x61, 0x6e, 0x52, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x2a, 0x3e, 0x0a, 0x0e, 0x45, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, - 0x10, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, - 0x45, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x41, 0x52, 0x43, 0x48, 0x49, 0x56, 0x45, 0x44, 0x10, 0x01, 0x42, 0xba, 0x01, 0x0a, 0x12, - 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, - 0x69, 0x6e, 0x42, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, - 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, - 0x67, 0x6f, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, - 0x6e, 0xa2, 0x02, 0x03, 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, - 0x64, 0x6c, 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, 0x1a, 0x46, 0x6c, 0x79, 0x74, - 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, - 0x6c, 0x3a, 0x3a, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x19, 0x0a, 0x17, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x76, 0x0a, 0x22, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x22, 0x4e, 0x0a, + 0x23, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x0a, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x53, 0x70, 0x61, 0x6e, 0x52, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x2a, 0x3e, 0x0a, + 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x14, 0x0a, 0x10, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x43, 0x54, + 0x49, 0x56, 0x45, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x41, 0x52, 0x43, 0x48, 0x49, 0x56, 0x45, 0x44, 0x10, 0x01, 0x42, 0xba, 0x01, + 0x0a, 0x12, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x42, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, + 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, + 0x6d, 0x69, 0x6e, 0xa2, 0x02, 0x03, 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, 0x46, 0x6c, 0x79, + 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, 0x1a, 0x46, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, 0x47, 0x50, 0x42, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, } var ( diff --git a/flyteidl/gen/pb-go/flyteidl/admin/schedule.pb.go b/flyteidl/gen/pb-go/flyteidl/admin/schedule.pb.go index d3a99e7a89..35e808c9ba 100644 --- a/flyteidl/gen/pb-go/flyteidl/admin/schedule.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/admin/schedule.pb.go @@ -70,6 +70,109 @@ func (FixedRateUnit) EnumDescriptor() ([]byte, []int) { return file_flyteidl_admin_schedule_proto_rawDescGZIP(), []int{0} } +type ConcurrencyPolicy int32 + +const ( + ConcurrencyPolicy_UNSPECIFIED ConcurrencyPolicy = 0 + // wait for previous executions to terminate before starting a new one + ConcurrencyPolicy_WAIT ConcurrencyPolicy = 1 + // fail the CreateExecution request and do not permit the execution to start + ConcurrencyPolicy_ABORT ConcurrencyPolicy = 2 + // terminate the oldest execution when the concurrency limit is reached and immediately begin proceeding with the new execution + ConcurrencyPolicy_REPLACE ConcurrencyPolicy = 3 +) + +// Enum value maps for ConcurrencyPolicy. +var ( + ConcurrencyPolicy_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "WAIT", + 2: "ABORT", + 3: "REPLACE", + } + ConcurrencyPolicy_value = map[string]int32{ + "UNSPECIFIED": 0, + "WAIT": 1, + "ABORT": 2, + "REPLACE": 3, + } +) + +func (x ConcurrencyPolicy) Enum() *ConcurrencyPolicy { + p := new(ConcurrencyPolicy) + *p = x + return p +} + +func (x ConcurrencyPolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ConcurrencyPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_flyteidl_admin_schedule_proto_enumTypes[1].Descriptor() +} + +func (ConcurrencyPolicy) Type() protoreflect.EnumType { + return &file_flyteidl_admin_schedule_proto_enumTypes[1] +} + +func (x ConcurrencyPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ConcurrencyPolicy.Descriptor instead. +func (ConcurrencyPolicy) EnumDescriptor() ([]byte, []int) { + return file_flyteidl_admin_schedule_proto_rawDescGZIP(), []int{1} +} + +type ConcurrencyLevel int32 + +const ( + // Applies concurrency limits across all launch plan versions. + ConcurrencyLevel_LAUNCH_PLAN ConcurrencyLevel = 0 + // Applies concurrency at the versioned launch plan level + ConcurrencyLevel_LAUNCH_PLAN_VERSION ConcurrencyLevel = 1 +) + +// Enum value maps for ConcurrencyLevel. +var ( + ConcurrencyLevel_name = map[int32]string{ + 0: "LAUNCH_PLAN", + 1: "LAUNCH_PLAN_VERSION", + } + ConcurrencyLevel_value = map[string]int32{ + "LAUNCH_PLAN": 0, + "LAUNCH_PLAN_VERSION": 1, + } +) + +func (x ConcurrencyLevel) Enum() *ConcurrencyLevel { + p := new(ConcurrencyLevel) + *p = x + return p +} + +func (x ConcurrencyLevel) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ConcurrencyLevel) Descriptor() protoreflect.EnumDescriptor { + return file_flyteidl_admin_schedule_proto_enumTypes[2].Descriptor() +} + +func (ConcurrencyLevel) Type() protoreflect.EnumType { + return &file_flyteidl_admin_schedule_proto_enumTypes[2] +} + +func (x ConcurrencyLevel) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ConcurrencyLevel.Descriptor instead. +func (ConcurrencyLevel) EnumDescriptor() ([]byte, []int) { + return file_flyteidl_admin_schedule_proto_rawDescGZIP(), []int{2} +} + // Option for schedules run at a certain frequency e.g. every 2 minutes. type FixedRate struct { state protoimpl.MessageState @@ -200,7 +303,8 @@ type Schedule struct { // *Schedule_CronSchedule ScheduleExpression isSchedule_ScheduleExpression `protobuf_oneof:"ScheduleExpression"` // Name of the input variable that the kickoff time will be supplied to when the workflow is kicked off. - KickoffTimeInputArg string `protobuf:"bytes,3,opt,name=kickoff_time_input_arg,json=kickoffTimeInputArg,proto3" json:"kickoff_time_input_arg,omitempty"` + KickoffTimeInputArg string `protobuf:"bytes,3,opt,name=kickoff_time_input_arg,json=kickoffTimeInputArg,proto3" json:"kickoff_time_input_arg,omitempty"` + SchedulerPolicy *SchedulerPolicy `protobuf:"bytes,5,opt,name=scheduler_policy,json=schedulerPolicy,proto3" json:"scheduler_policy,omitempty"` } func (x *Schedule) Reset() { @@ -271,6 +375,13 @@ func (x *Schedule) GetKickoffTimeInputArg() string { return "" } +func (x *Schedule) GetSchedulerPolicy() *SchedulerPolicy { + if x != nil { + return x.SchedulerPolicy + } + return nil +} + type isSchedule_ScheduleExpression interface { isSchedule_ScheduleExpression() } @@ -297,6 +408,63 @@ func (*Schedule_Rate) isSchedule_ScheduleExpression() {} func (*Schedule_CronSchedule) isSchedule_ScheduleExpression() {} +type SchedulerPolicy struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Defines how many executions with this launch plan can run in parallel + Max uint32 `protobuf:"varint,1,opt,name=max,proto3" json:"max,omitempty"` + // Defines how to handle the execution when the max concurrency is reached. + Policy ConcurrencyPolicy `protobuf:"varint,2,opt,name=policy,proto3,enum=flyteidl.admin.ConcurrencyPolicy" json:"policy,omitempty"` +} + +func (x *SchedulerPolicy) Reset() { + *x = SchedulerPolicy{} + if protoimpl.UnsafeEnabled { + mi := &file_flyteidl_admin_schedule_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SchedulerPolicy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SchedulerPolicy) ProtoMessage() {} + +func (x *SchedulerPolicy) ProtoReflect() protoreflect.Message { + mi := &file_flyteidl_admin_schedule_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 SchedulerPolicy.ProtoReflect.Descriptor instead. +func (*SchedulerPolicy) Descriptor() ([]byte, []int) { + return file_flyteidl_admin_schedule_proto_rawDescGZIP(), []int{3} +} + +func (x *SchedulerPolicy) GetMax() uint32 { + if x != nil { + return x.Max + } + return 0 +} + +func (x *SchedulerPolicy) GetPolicy() ConcurrencyPolicy { + if x != nil { + return x.Policy + } + return ConcurrencyPolicy_UNSPECIFIED +} + var File_flyteidl_admin_schedule_proto protoreflect.FileDescriptor var file_flyteidl_admin_schedule_proto_rawDesc = []byte{ @@ -312,7 +480,7 @@ var file_flyteidl_admin_schedule_proto_rawDesc = []byte{ 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x08, 0x53, 0x63, + 0x09, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x22, 0xc6, 0x02, 0x0a, 0x08, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x2d, 0x0a, 0x0f, 0x63, 0x72, 0x6f, 0x6e, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x72, 0x6f, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x65, @@ -327,23 +495,42 @@ var file_flyteidl_admin_schedule_proto_rawDesc = []byte{ 0x69, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x61, 0x72, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x6b, 0x69, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x54, 0x69, 0x6d, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x41, 0x72, 0x67, - 0x42, 0x14, 0x0a, 0x12, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x45, 0x78, 0x70, 0x72, - 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2a, 0x2e, 0x0a, 0x0d, 0x46, 0x69, 0x78, 0x65, 0x64, 0x52, - 0x61, 0x74, 0x65, 0x55, 0x6e, 0x69, 0x74, 0x12, 0x0a, 0x0a, 0x06, 0x4d, 0x49, 0x4e, 0x55, 0x54, - 0x45, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x01, 0x12, 0x07, 0x0a, - 0x03, 0x44, 0x41, 0x59, 0x10, 0x02, 0x42, 0xb9, 0x01, 0x0a, 0x12, 0x63, 0x6f, 0x6d, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x42, 0x0d, 0x53, - 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, - 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, - 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, 0x02, 0x03, 0x46, 0x41, - 0x58, 0xaa, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x41, 0x64, 0x6d, - 0x69, 0x6e, 0xca, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, - 0x6d, 0x69, 0x6e, 0xe2, 0x02, 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, - 0x64, 0x6d, 0x69, 0x6e, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0xea, 0x02, 0x0f, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x41, 0x64, 0x6d, - 0x69, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, + 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x53, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0f, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x14, 0x0a, 0x12, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x22, 0x5e, 0x0a, 0x0f, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x61, 0x78, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x03, 0x6d, 0x61, 0x78, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, + 0x65, 0x6e, 0x63, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x06, 0x70, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x2a, 0x2e, 0x0a, 0x0d, 0x46, 0x69, 0x78, 0x65, 0x64, 0x52, 0x61, 0x74, 0x65, 0x55, + 0x6e, 0x69, 0x74, 0x12, 0x0a, 0x0a, 0x06, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x10, 0x00, 0x12, + 0x08, 0x0a, 0x04, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x41, 0x59, + 0x10, 0x02, 0x2a, 0x46, 0x0a, 0x11, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, + 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x57, 0x41, 0x49, 0x54, + 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x10, 0x02, 0x12, 0x0b, 0x0a, + 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x03, 0x2a, 0x3c, 0x0a, 0x10, 0x43, 0x6f, + 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x0f, + 0x0a, 0x0b, 0x4c, 0x41, 0x55, 0x4e, 0x43, 0x48, 0x5f, 0x50, 0x4c, 0x41, 0x4e, 0x10, 0x00, 0x12, + 0x17, 0x0a, 0x13, 0x4c, 0x41, 0x55, 0x4e, 0x43, 0x48, 0x5f, 0x50, 0x4c, 0x41, 0x4e, 0x5f, 0x56, + 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x42, 0xb9, 0x01, 0x0a, 0x12, 0x63, 0x6f, 0x6d, + 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x42, + 0x0d, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, + 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, + 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, 0x02, 0x03, + 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x41, + 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, + 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, + 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x41, + 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -358,23 +545,28 @@ func file_flyteidl_admin_schedule_proto_rawDescGZIP() []byte { return file_flyteidl_admin_schedule_proto_rawDescData } -var file_flyteidl_admin_schedule_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_flyteidl_admin_schedule_proto_msgTypes = make([]protoimpl.MessageInfo, 3) +var file_flyteidl_admin_schedule_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_flyteidl_admin_schedule_proto_msgTypes = make([]protoimpl.MessageInfo, 4) var file_flyteidl_admin_schedule_proto_goTypes = []interface{}{ - (FixedRateUnit)(0), // 0: flyteidl.admin.FixedRateUnit - (*FixedRate)(nil), // 1: flyteidl.admin.FixedRate - (*CronSchedule)(nil), // 2: flyteidl.admin.CronSchedule - (*Schedule)(nil), // 3: flyteidl.admin.Schedule + (FixedRateUnit)(0), // 0: flyteidl.admin.FixedRateUnit + (ConcurrencyPolicy)(0), // 1: flyteidl.admin.ConcurrencyPolicy + (ConcurrencyLevel)(0), // 2: flyteidl.admin.ConcurrencyLevel + (*FixedRate)(nil), // 3: flyteidl.admin.FixedRate + (*CronSchedule)(nil), // 4: flyteidl.admin.CronSchedule + (*Schedule)(nil), // 5: flyteidl.admin.Schedule + (*SchedulerPolicy)(nil), // 6: flyteidl.admin.SchedulerPolicy } var file_flyteidl_admin_schedule_proto_depIdxs = []int32{ 0, // 0: flyteidl.admin.FixedRate.unit:type_name -> flyteidl.admin.FixedRateUnit - 1, // 1: flyteidl.admin.Schedule.rate:type_name -> flyteidl.admin.FixedRate - 2, // 2: flyteidl.admin.Schedule.cron_schedule:type_name -> flyteidl.admin.CronSchedule - 3, // [3:3] is the sub-list for method output_type - 3, // [3:3] is the sub-list for method input_type - 3, // [3:3] is the sub-list for extension type_name - 3, // [3:3] is the sub-list for extension extendee - 0, // [0:3] is the sub-list for field type_name + 3, // 1: flyteidl.admin.Schedule.rate:type_name -> flyteidl.admin.FixedRate + 4, // 2: flyteidl.admin.Schedule.cron_schedule:type_name -> flyteidl.admin.CronSchedule + 6, // 3: flyteidl.admin.Schedule.scheduler_policy:type_name -> flyteidl.admin.SchedulerPolicy + 1, // 4: flyteidl.admin.SchedulerPolicy.policy:type_name -> flyteidl.admin.ConcurrencyPolicy + 5, // [5:5] is the sub-list for method output_type + 5, // [5:5] is the sub-list for method input_type + 5, // [5:5] is the sub-list for extension type_name + 5, // [5:5] is the sub-list for extension extendee + 0, // [0:5] is the sub-list for field type_name } func init() { file_flyteidl_admin_schedule_proto_init() } @@ -419,6 +611,18 @@ func file_flyteidl_admin_schedule_proto_init() { return nil } } + file_flyteidl_admin_schedule_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SchedulerPolicy); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_flyteidl_admin_schedule_proto_msgTypes[2].OneofWrappers = []interface{}{ (*Schedule_CronExpression)(nil), @@ -430,8 +634,8 @@ func file_flyteidl_admin_schedule_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flyteidl_admin_schedule_proto_rawDesc, - NumEnums: 1, - NumMessages: 3, + NumEnums: 3, + NumMessages: 4, NumExtensions: 0, NumServices: 0, }, diff --git a/flyteidl/gen/pb-go/flyteidl/core/execution.pb.go b/flyteidl/gen/pb-go/flyteidl/core/execution.pb.go index a17e94eba1..2efe87818a 100644 --- a/flyteidl/gen/pb-go/flyteidl/core/execution.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/core/execution.pb.go @@ -35,21 +35,23 @@ const ( WorkflowExecution_ABORTED WorkflowExecution_Phase = 7 WorkflowExecution_TIMED_OUT WorkflowExecution_Phase = 8 WorkflowExecution_ABORTING WorkflowExecution_Phase = 9 + WorkflowExecution_PENDING WorkflowExecution_Phase = 10 ) // Enum value maps for WorkflowExecution_Phase. var ( WorkflowExecution_Phase_name = map[int32]string{ - 0: "UNDEFINED", - 1: "QUEUED", - 2: "RUNNING", - 3: "SUCCEEDING", - 4: "SUCCEEDED", - 5: "FAILING", - 6: "FAILED", - 7: "ABORTED", - 8: "TIMED_OUT", - 9: "ABORTING", + 0: "UNDEFINED", + 1: "QUEUED", + 2: "RUNNING", + 3: "SUCCEEDING", + 4: "SUCCEEDED", + 5: "FAILING", + 6: "FAILED", + 7: "ABORTED", + 8: "TIMED_OUT", + 9: "ABORTING", + 10: "PENDING", } WorkflowExecution_Phase_value = map[string]int32{ "UNDEFINED": 0, @@ -62,6 +64,7 @@ var ( "ABORTED": 7, "TIMED_OUT": 8, "ABORTING": 9, + "PENDING": 10, } ) @@ -825,8 +828,8 @@ var file_flyteidl_core_execution_proto_rawDesc = []byte{ 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, - 0xa7, 0x01, 0x0a, 0x11, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x91, 0x01, 0x0a, 0x05, 0x50, 0x68, 0x61, 0x73, 0x65, 0x12, + 0xb4, 0x01, 0x0a, 0x11, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x9e, 0x01, 0x0a, 0x05, 0x50, 0x68, 0x61, 0x73, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x51, 0x55, 0x45, 0x55, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x53, 0x55, 0x43, 0x43, 0x45, @@ -835,96 +838,97 @@ var file_flyteidl_core_execution_proto_rawDesc = []byte{ 0x47, 0x10, 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x41, - 0x42, 0x4f, 0x52, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x09, 0x22, 0xb6, 0x01, 0x0a, 0x0d, 0x4e, 0x6f, - 0x64, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa4, 0x01, 0x0a, 0x05, - 0x50, 0x68, 0x61, 0x73, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, - 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x51, 0x55, 0x45, 0x55, 0x45, 0x44, 0x10, 0x01, - 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0d, 0x0a, - 0x09, 0x53, 0x55, 0x43, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, - 0x46, 0x41, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, - 0x4c, 0x45, 0x44, 0x10, 0x05, 0x12, 0x0b, 0x0a, 0x07, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x45, 0x44, - 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x4b, 0x49, 0x50, 0x50, 0x45, 0x44, 0x10, 0x07, 0x12, - 0x0d, 0x0a, 0x09, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x08, 0x12, 0x13, - 0x0a, 0x0f, 0x44, 0x59, 0x4e, 0x41, 0x4d, 0x49, 0x43, 0x5f, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, - 0x47, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, 0x45, 0x44, - 0x10, 0x0a, 0x22, 0x96, 0x01, 0x0a, 0x0d, 0x54, 0x61, 0x73, 0x6b, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x84, 0x01, 0x0a, 0x05, 0x50, 0x68, 0x61, 0x73, 0x65, 0x12, 0x0d, - 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, - 0x06, 0x51, 0x55, 0x45, 0x55, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x55, 0x4e, - 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x43, 0x43, 0x45, 0x45, - 0x44, 0x45, 0x44, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x45, 0x44, - 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x05, 0x12, 0x10, - 0x0a, 0x0c, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x49, 0x4e, 0x47, 0x10, 0x06, - 0x12, 0x19, 0x0a, 0x15, 0x57, 0x41, 0x49, 0x54, 0x49, 0x4e, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x5f, - 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x53, 0x10, 0x07, 0x22, 0x9a, 0x02, 0x0a, 0x0e, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, - 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, - 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1b, 0x0a, 0x09, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x55, 0x72, 0x69, 0x12, 0x3b, 0x0a, 0x04, 0x6b, 0x69, 0x6e, - 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, - 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4b, 0x69, 0x6e, 0x64, - 0x52, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x12, 0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x12, 0x16, 0x0a, 0x06, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x06, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x22, 0x2e, 0x0a, 0x09, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, - 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x55, 0x53, 0x45, 0x52, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, - 0x53, 0x59, 0x53, 0x54, 0x45, 0x4d, 0x10, 0x02, 0x22, 0xb2, 0x02, 0x0a, 0x07, 0x54, 0x61, 0x73, - 0x6b, 0x4c, 0x6f, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x69, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x75, 0x72, 0x69, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x4b, 0x0a, 0x0e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x4c, 0x6f, 0x67, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x52, 0x0d, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x2b, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x03, 0x74, 0x74, 0x6c, 0x12, 0x2a, 0x0a, 0x10, 0x53, 0x68, 0x6f, 0x77, 0x57, 0x68, 0x69, 0x6c, - 0x65, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, - 0x53, 0x68, 0x6f, 0x77, 0x57, 0x68, 0x69, 0x6c, 0x65, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, - 0x12, 0x2a, 0x0a, 0x10, 0x48, 0x69, 0x64, 0x65, 0x4f, 0x6e, 0x63, 0x65, 0x46, 0x69, 0x6e, 0x69, - 0x73, 0x68, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x48, 0x69, 0x64, 0x65, - 0x4f, 0x6e, 0x63, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x22, 0x2f, 0x0a, 0x0d, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x0b, 0x0a, - 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x43, 0x53, - 0x56, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x4a, 0x53, 0x4f, 0x4e, 0x10, 0x02, 0x22, 0x5a, 0x0a, - 0x14, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x42, 0x0a, 0x0f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x69, 0x6e, - 0x67, 0x5f, 0x62, 0x75, 0x64, 0x67, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, - 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x71, 0x75, 0x65, 0x75, 0x65, - 0x69, 0x6e, 0x67, 0x42, 0x75, 0x64, 0x67, 0x65, 0x74, 0x22, 0xce, 0x01, 0x0a, 0x10, 0x51, 0x75, - 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x3a, - 0x0a, 0x04, 0x74, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, - 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x54, 0x69, - 0x65, 0x72, 0x48, 0x00, 0x52, 0x04, 0x74, 0x69, 0x65, 0x72, 0x12, 0x39, 0x0a, 0x04, 0x73, 0x70, - 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, - 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, - 0x04, 0x73, 0x70, 0x65, 0x63, 0x22, 0x34, 0x0a, 0x04, 0x54, 0x69, 0x65, 0x72, 0x12, 0x0d, 0x0a, - 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, - 0x48, 0x49, 0x47, 0x48, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4d, 0x45, 0x44, 0x49, 0x55, 0x4d, - 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x4f, 0x57, 0x10, 0x03, 0x42, 0x0d, 0x0a, 0x0b, 0x64, - 0x65, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0xb4, 0x01, 0x0a, 0x11, 0x63, - 0x6f, 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, - 0x42, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, - 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0xa2, 0x02, - 0x03, 0x46, 0x43, 0x58, 0xaa, 0x02, 0x0d, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, - 0x43, 0x6f, 0x72, 0x65, 0xca, 0x02, 0x0d, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, - 0x43, 0x6f, 0x72, 0x65, 0xe2, 0x02, 0x19, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, - 0x43, 0x6f, 0x72, 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0xea, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x43, 0x6f, 0x72, - 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x42, 0x4f, 0x52, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x09, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x45, 0x4e, + 0x44, 0x49, 0x4e, 0x47, 0x10, 0x0a, 0x22, 0xb6, 0x01, 0x0a, 0x0d, 0x4e, 0x6f, 0x64, 0x65, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa4, 0x01, 0x0a, 0x05, 0x50, 0x68, 0x61, + 0x73, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, + 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x51, 0x55, 0x45, 0x55, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, + 0x07, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, + 0x43, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x41, 0x49, + 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, + 0x10, 0x05, 0x12, 0x0b, 0x0a, 0x07, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0x06, 0x12, + 0x0b, 0x0a, 0x07, 0x53, 0x4b, 0x49, 0x50, 0x50, 0x45, 0x44, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, + 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x08, 0x12, 0x13, 0x0a, 0x0f, 0x44, + 0x59, 0x4e, 0x41, 0x4d, 0x49, 0x43, 0x5f, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x09, + 0x12, 0x0d, 0x0a, 0x09, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, 0x45, 0x44, 0x10, 0x0a, 0x22, + 0x96, 0x01, 0x0a, 0x0d, 0x54, 0x61, 0x73, 0x6b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x22, 0x84, 0x01, 0x0a, 0x05, 0x50, 0x68, 0x61, 0x73, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x55, + 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x51, 0x55, + 0x45, 0x55, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, + 0x47, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x43, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, + 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0x04, 0x12, + 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x05, 0x12, 0x10, 0x0a, 0x0c, 0x49, + 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x49, 0x4e, 0x47, 0x10, 0x06, 0x12, 0x19, 0x0a, + 0x15, 0x57, 0x41, 0x49, 0x54, 0x49, 0x4e, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x5f, 0x52, 0x45, 0x53, + 0x4f, 0x55, 0x52, 0x43, 0x45, 0x53, 0x10, 0x07, 0x22, 0x9a, 0x02, 0x0a, 0x0e, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x63, + 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, + 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x55, 0x72, 0x69, 0x12, 0x3b, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, + 0x72, 0x6f, 0x72, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4b, 0x69, 0x6e, 0x64, 0x52, 0x04, 0x6b, + 0x69, 0x6e, 0x64, 0x12, 0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x16, 0x0a, + 0x06, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x77, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x22, 0x2e, 0x0a, 0x09, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4b, 0x69, + 0x6e, 0x64, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, + 0x08, 0x0a, 0x04, 0x55, 0x53, 0x45, 0x52, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x59, 0x53, + 0x54, 0x45, 0x4d, 0x10, 0x02, 0x22, 0xb2, 0x02, 0x0a, 0x07, 0x54, 0x61, 0x73, 0x6b, 0x4c, 0x6f, + 0x67, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x69, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x75, 0x72, 0x69, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x4b, 0x0a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x5f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x24, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, + 0x54, 0x61, 0x73, 0x6b, 0x4c, 0x6f, 0x67, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x46, + 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x52, 0x0d, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x46, 0x6f, + 0x72, 0x6d, 0x61, 0x74, 0x12, 0x2b, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x74, 0x74, + 0x6c, 0x12, 0x2a, 0x0a, 0x10, 0x53, 0x68, 0x6f, 0x77, 0x57, 0x68, 0x69, 0x6c, 0x65, 0x50, 0x65, + 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x53, 0x68, 0x6f, + 0x77, 0x57, 0x68, 0x69, 0x6c, 0x65, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x12, 0x2a, 0x0a, + 0x10, 0x48, 0x69, 0x64, 0x65, 0x4f, 0x6e, 0x63, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, + 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x48, 0x69, 0x64, 0x65, 0x4f, 0x6e, 0x63, + 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x22, 0x2f, 0x0a, 0x0d, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, + 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x43, 0x53, 0x56, 0x10, 0x01, + 0x12, 0x08, 0x0a, 0x04, 0x4a, 0x53, 0x4f, 0x4e, 0x10, 0x02, 0x22, 0x5a, 0x0a, 0x14, 0x51, 0x75, + 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x70, + 0x65, 0x63, 0x12, 0x42, 0x0a, 0x0f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x69, 0x6e, 0x67, 0x5f, 0x62, + 0x75, 0x64, 0x67, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x69, 0x6e, 0x67, + 0x42, 0x75, 0x64, 0x67, 0x65, 0x74, 0x22, 0xce, 0x01, 0x0a, 0x10, 0x51, 0x75, 0x61, 0x6c, 0x69, + 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x3a, 0x0a, 0x04, 0x74, + 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, + 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x54, 0x69, 0x65, 0x72, 0x48, + 0x00, 0x52, 0x04, 0x74, 0x69, 0x65, 0x72, 0x12, 0x39, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, + 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x04, 0x73, 0x70, + 0x65, 0x63, 0x22, 0x34, 0x0a, 0x04, 0x54, 0x69, 0x65, 0x72, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, + 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x48, 0x49, 0x47, + 0x48, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4d, 0x45, 0x44, 0x49, 0x55, 0x4d, 0x10, 0x02, 0x12, + 0x07, 0x0a, 0x03, 0x4c, 0x4f, 0x57, 0x10, 0x03, 0x42, 0x0d, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x69, + 0x67, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0xb4, 0x01, 0x0a, 0x11, 0x63, 0x6f, 0x6d, 0x2e, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x42, 0x0e, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, + 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0xa2, 0x02, 0x03, 0x46, 0x43, + 0x58, 0xaa, 0x02, 0x0d, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x43, 0x6f, 0x72, + 0x65, 0xca, 0x02, 0x0d, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x43, 0x6f, 0x72, + 0x65, 0xe2, 0x02, 0x19, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x43, 0x6f, 0x72, + 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0e, + 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x43, 0x6f, 0x72, 0x65, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json b/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json index 52fbff78d6..07c00867b7 100644 --- a/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json +++ b/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json @@ -4580,6 +4580,17 @@ } } }, + "adminConcurrencyPolicy": { + "type": "string", + "enum": [ + "UNSPECIFIED", + "WAIT", + "ABORT", + "REPLACE" + ], + "default": "UNSPECIFIED", + "title": "- WAIT: wait for previous executions to terminate before starting a new one\n - ABORT: fail the CreateExecution request and do not permit the execution to start\n - REPLACE: terminate the oldest execution when the concurrency limit is reached and immediately begin proceeding with the new execution" + }, "adminCronSchedule": { "type": "object", "properties": { @@ -5095,6 +5106,10 @@ "principal": { "type": "string", "title": "Identifies the entity (if any) responsible for causing the state change of the execution" + }, + "description": { + "type": "string", + "title": "Includes the reason for the `PENDING` phase" } } }, @@ -5989,10 +6004,27 @@ "kickoff_time_input_arg": { "type": "string", "description": "Name of the input variable that the kickoff time will be supplied to when the workflow is kicked off." + }, + "scheduler_policy": { + "$ref": "#/definitions/adminSchedulerPolicy" } }, "description": "Defines complete set of information required to trigger an execution on a schedule." }, + "adminSchedulerPolicy": { + "type": "object", + "properties": { + "max": { + "type": "integer", + "format": "int64", + "title": "Defines how many executions with this launch plan can run in parallel" + }, + "policy": { + "$ref": "#/definitions/adminConcurrencyPolicy", + "description": "Defines how to handle the execution when the max concurrency is reached." + } + } + }, "adminSlackNotification": { "type": "object", "properties": { @@ -8611,7 +8643,8 @@ "FAILED", "ABORTED", "TIMED_OUT", - "ABORTING" + "ABORTING", + "PENDING" ], "default": "UNDEFINED" }, diff --git a/flyteidl/gen/pb-js/flyteidl.d.ts b/flyteidl/gen/pb-js/flyteidl.d.ts index 9b4f61dbde..fd3e287508 100644 --- a/flyteidl/gen/pb-js/flyteidl.d.ts +++ b/flyteidl/gen/pb-js/flyteidl.d.ts @@ -5647,7 +5647,8 @@ export namespace flyteidl { FAILED = 6, ABORTED = 7, TIMED_OUT = 8, - ABORTING = 9 + ABORTING = 9, + PENDING = 10 } } @@ -14878,6 +14879,9 @@ export namespace flyteidl { /** ExecutionStateChangeDetails principal */ principal?: (string|null); + + /** ExecutionStateChangeDetails description */ + description?: (string|null); } /** Represents an ExecutionStateChangeDetails. */ @@ -14898,6 +14902,9 @@ export namespace flyteidl { /** ExecutionStateChangeDetails principal. */ public principal: string; + /** ExecutionStateChangeDetails description. */ + public description: string; + /** * Creates a new ExecutionStateChangeDetails instance using the specified properties. * @param [properties] Properties to set @@ -16862,6 +16869,9 @@ export namespace flyteidl { /** Schedule kickoffTimeInputArg */ kickoffTimeInputArg?: (string|null); + + /** Schedule schedulerPolicy */ + schedulerPolicy?: (flyteidl.admin.ISchedulerPolicy|null); } /** Represents a Schedule. */ @@ -16885,6 +16895,9 @@ export namespace flyteidl { /** Schedule kickoffTimeInputArg. */ public kickoffTimeInputArg: string; + /** Schedule schedulerPolicy. */ + public schedulerPolicy?: (flyteidl.admin.ISchedulerPolicy|null); + /** Schedule ScheduleExpression. */ public ScheduleExpression?: ("cronExpression"|"rate"|"cronSchedule"); @@ -16921,6 +16934,78 @@ export namespace flyteidl { public static verify(message: { [k: string]: any }): (string|null); } + /** Properties of a SchedulerPolicy. */ + interface ISchedulerPolicy { + + /** SchedulerPolicy max */ + max?: (number|null); + + /** SchedulerPolicy policy */ + policy?: (flyteidl.admin.ConcurrencyPolicy|null); + } + + /** Represents a SchedulerPolicy. */ + class SchedulerPolicy implements ISchedulerPolicy { + + /** + * Constructs a new SchedulerPolicy. + * @param [properties] Properties to set + */ + constructor(properties?: flyteidl.admin.ISchedulerPolicy); + + /** SchedulerPolicy max. */ + public max: number; + + /** SchedulerPolicy policy. */ + public policy: flyteidl.admin.ConcurrencyPolicy; + + /** + * Creates a new SchedulerPolicy instance using the specified properties. + * @param [properties] Properties to set + * @returns SchedulerPolicy instance + */ + public static create(properties?: flyteidl.admin.ISchedulerPolicy): flyteidl.admin.SchedulerPolicy; + + /** + * Encodes the specified SchedulerPolicy message. Does not implicitly {@link flyteidl.admin.SchedulerPolicy.verify|verify} messages. + * @param message SchedulerPolicy message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: flyteidl.admin.ISchedulerPolicy, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a SchedulerPolicy message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns SchedulerPolicy + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): flyteidl.admin.SchedulerPolicy; + + /** + * Verifies a SchedulerPolicy message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + } + + /** ConcurrencyPolicy enum. */ + enum ConcurrencyPolicy { + UNSPECIFIED = 0, + WAIT = 1, + ABORT = 2, + REPLACE = 3 + } + + /** ConcurrencyLevel enum. */ + enum ConcurrencyLevel { + LAUNCH_PLAN = 0, + LAUNCH_PLAN_VERSION = 1 + } + /** Properties of a NodeExecutionGetRequest. */ interface INodeExecutionGetRequest { diff --git a/flyteidl/gen/pb-js/flyteidl.js b/flyteidl/gen/pb-js/flyteidl.js index bbc579a293..ebd4187e32 100644 --- a/flyteidl/gen/pb-js/flyteidl.js +++ b/flyteidl/gen/pb-js/flyteidl.js @@ -13702,6 +13702,7 @@ * @property {number} ABORTED=7 ABORTED value * @property {number} TIMED_OUT=8 TIMED_OUT value * @property {number} ABORTING=9 ABORTING value + * @property {number} PENDING=10 PENDING value */ WorkflowExecution.Phase = (function() { var valuesById = {}, values = Object.create(valuesById); @@ -13715,6 +13716,7 @@ values[valuesById[7] = "ABORTED"] = 7; values[valuesById[8] = "TIMED_OUT"] = 8; values[valuesById[9] = "ABORTING"] = 9; + values[valuesById[10] = "PENDING"] = 10; return values; })(); @@ -20982,6 +20984,7 @@ case 7: case 8: case 9: + case 10: break; } if (message.occurredAt != null && message.hasOwnProperty("occurredAt")) { @@ -30054,6 +30057,7 @@ case 7: case 8: case 9: + case 10: break; } } @@ -34504,6 +34508,7 @@ case 7: case 8: case 9: + case 10: break; } if (message.startedAt != null && message.hasOwnProperty("startedAt")) { @@ -36223,6 +36228,7 @@ * @property {flyteidl.admin.ExecutionState|null} [state] ExecutionStateChangeDetails state * @property {google.protobuf.ITimestamp|null} [occurredAt] ExecutionStateChangeDetails occurredAt * @property {string|null} [principal] ExecutionStateChangeDetails principal + * @property {string|null} [description] ExecutionStateChangeDetails description */ /** @@ -36264,6 +36270,14 @@ */ ExecutionStateChangeDetails.prototype.principal = ""; + /** + * ExecutionStateChangeDetails description. + * @member {string} description + * @memberof flyteidl.admin.ExecutionStateChangeDetails + * @instance + */ + ExecutionStateChangeDetails.prototype.description = ""; + /** * Creates a new ExecutionStateChangeDetails instance using the specified properties. * @function create @@ -36294,6 +36308,8 @@ $root.google.protobuf.Timestamp.encode(message.occurredAt, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.principal != null && message.hasOwnProperty("principal")) writer.uint32(/* id 3, wireType 2 =*/26).string(message.principal); + if (message.description != null && message.hasOwnProperty("description")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.description); return writer; }; @@ -36324,6 +36340,9 @@ case 3: message.principal = reader.string(); break; + case 4: + message.description = reader.string(); + break; default: reader.skipType(tag & 7); break; @@ -36359,6 +36378,9 @@ if (message.principal != null && message.hasOwnProperty("principal")) if (!$util.isString(message.principal)) return "principal: string expected"; + if (message.description != null && message.hasOwnProperty("description")) + if (!$util.isString(message.description)) + return "description: string expected"; return null; }; @@ -40956,6 +40978,7 @@ * @property {flyteidl.admin.IFixedRate|null} [rate] Schedule rate * @property {flyteidl.admin.ICronSchedule|null} [cronSchedule] Schedule cronSchedule * @property {string|null} [kickoffTimeInputArg] Schedule kickoffTimeInputArg + * @property {flyteidl.admin.ISchedulerPolicy|null} [schedulerPolicy] Schedule schedulerPolicy */ /** @@ -41005,6 +41028,14 @@ */ Schedule.prototype.kickoffTimeInputArg = ""; + /** + * Schedule schedulerPolicy. + * @member {flyteidl.admin.ISchedulerPolicy|null|undefined} schedulerPolicy + * @memberof flyteidl.admin.Schedule + * @instance + */ + Schedule.prototype.schedulerPolicy = null; + // OneOf field names bound to virtual getters and setters var $oneOfFields; @@ -41051,6 +41082,8 @@ writer.uint32(/* id 3, wireType 2 =*/26).string(message.kickoffTimeInputArg); if (message.cronSchedule != null && message.hasOwnProperty("cronSchedule")) $root.flyteidl.admin.CronSchedule.encode(message.cronSchedule, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + if (message.schedulerPolicy != null && message.hasOwnProperty("schedulerPolicy")) + $root.flyteidl.admin.SchedulerPolicy.encode(message.schedulerPolicy, writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); return writer; }; @@ -41084,6 +41117,9 @@ case 3: message.kickoffTimeInputArg = reader.string(); break; + case 5: + message.schedulerPolicy = $root.flyteidl.admin.SchedulerPolicy.decode(reader, reader.uint32()); + break; default: reader.skipType(tag & 7); break; @@ -41132,12 +41168,183 @@ if (message.kickoffTimeInputArg != null && message.hasOwnProperty("kickoffTimeInputArg")) if (!$util.isString(message.kickoffTimeInputArg)) return "kickoffTimeInputArg: string expected"; + if (message.schedulerPolicy != null && message.hasOwnProperty("schedulerPolicy")) { + var error = $root.flyteidl.admin.SchedulerPolicy.verify(message.schedulerPolicy); + if (error) + return "schedulerPolicy." + error; + } return null; }; return Schedule; })(); + admin.SchedulerPolicy = (function() { + + /** + * Properties of a SchedulerPolicy. + * @memberof flyteidl.admin + * @interface ISchedulerPolicy + * @property {number|null} [max] SchedulerPolicy max + * @property {flyteidl.admin.ConcurrencyPolicy|null} [policy] SchedulerPolicy policy + */ + + /** + * Constructs a new SchedulerPolicy. + * @memberof flyteidl.admin + * @classdesc Represents a SchedulerPolicy. + * @implements ISchedulerPolicy + * @constructor + * @param {flyteidl.admin.ISchedulerPolicy=} [properties] Properties to set + */ + function SchedulerPolicy(properties) { + if (properties) + for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * SchedulerPolicy max. + * @member {number} max + * @memberof flyteidl.admin.SchedulerPolicy + * @instance + */ + SchedulerPolicy.prototype.max = 0; + + /** + * SchedulerPolicy policy. + * @member {flyteidl.admin.ConcurrencyPolicy} policy + * @memberof flyteidl.admin.SchedulerPolicy + * @instance + */ + SchedulerPolicy.prototype.policy = 0; + + /** + * Creates a new SchedulerPolicy instance using the specified properties. + * @function create + * @memberof flyteidl.admin.SchedulerPolicy + * @static + * @param {flyteidl.admin.ISchedulerPolicy=} [properties] Properties to set + * @returns {flyteidl.admin.SchedulerPolicy} SchedulerPolicy instance + */ + SchedulerPolicy.create = function create(properties) { + return new SchedulerPolicy(properties); + }; + + /** + * Encodes the specified SchedulerPolicy message. Does not implicitly {@link flyteidl.admin.SchedulerPolicy.verify|verify} messages. + * @function encode + * @memberof flyteidl.admin.SchedulerPolicy + * @static + * @param {flyteidl.admin.ISchedulerPolicy} message SchedulerPolicy message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + SchedulerPolicy.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.max != null && message.hasOwnProperty("max")) + writer.uint32(/* id 1, wireType 0 =*/8).uint32(message.max); + if (message.policy != null && message.hasOwnProperty("policy")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.policy); + return writer; + }; + + /** + * Decodes a SchedulerPolicy message from the specified reader or buffer. + * @function decode + * @memberof flyteidl.admin.SchedulerPolicy + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {flyteidl.admin.SchedulerPolicy} SchedulerPolicy + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + SchedulerPolicy.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + var end = length === undefined ? reader.len : reader.pos + length, message = new $root.flyteidl.admin.SchedulerPolicy(); + while (reader.pos < end) { + var tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + message.max = reader.uint32(); + break; + case 2: + message.policy = reader.int32(); + break; + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Verifies a SchedulerPolicy message. + * @function verify + * @memberof flyteidl.admin.SchedulerPolicy + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + SchedulerPolicy.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.max != null && message.hasOwnProperty("max")) + if (!$util.isInteger(message.max)) + return "max: integer expected"; + if (message.policy != null && message.hasOwnProperty("policy")) + switch (message.policy) { + default: + return "policy: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } + return null; + }; + + return SchedulerPolicy; + })(); + + /** + * ConcurrencyPolicy enum. + * @name flyteidl.admin.ConcurrencyPolicy + * @enum {string} + * @property {number} UNSPECIFIED=0 UNSPECIFIED value + * @property {number} WAIT=1 WAIT value + * @property {number} ABORT=2 ABORT value + * @property {number} REPLACE=3 REPLACE value + */ + admin.ConcurrencyPolicy = (function() { + var valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "UNSPECIFIED"] = 0; + values[valuesById[1] = "WAIT"] = 1; + values[valuesById[2] = "ABORT"] = 2; + values[valuesById[3] = "REPLACE"] = 3; + return values; + })(); + + /** + * ConcurrencyLevel enum. + * @name flyteidl.admin.ConcurrencyLevel + * @enum {string} + * @property {number} LAUNCH_PLAN=0 LAUNCH_PLAN value + * @property {number} LAUNCH_PLAN_VERSION=1 LAUNCH_PLAN_VERSION value + */ + admin.ConcurrencyLevel = (function() { + var valuesById = {}, values = Object.create(valuesById); + values[valuesById[0] = "LAUNCH_PLAN"] = 0; + values[valuesById[1] = "LAUNCH_PLAN_VERSION"] = 1; + return values; + })(); + admin.NodeExecutionGetRequest = (function() { /** diff --git a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py index d766ba6ee9..7b4a50cfc5 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py @@ -26,7 +26,7 @@ from flyteidl.admin import matchable_resource_pb2 as flyteidl_dot_admin_dot_matchable__resource__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1e\x66lyteidl/admin/execution.proto\x12\x0e\x66lyteidl.admin\x1a\'flyteidl/admin/cluster_assignment.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\"flyteidl/core/execution_envs.proto\x1a\x1f\x66lyteidl/core/artifact_id.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1b\x66lyteidl/core/metrics.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\x1a\'flyteidl/admin/matchable_resource.proto\"\xd6\x01\n\x16\x45xecutionCreateRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\x31\n\x04spec\x18\x04 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12\x31\n\x06inputs\x18\x05 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x06inputs\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org\"\x99\x01\n\x18\x45xecutionRelaunchRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\'\n\x0foverwrite_cache\x18\x04 \x01(\x08R\x0eoverwriteCacheJ\x04\x08\x02\x10\x03\"\xa8\x01\n\x17\x45xecutionRecoverRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\"U\n\x17\x45xecutionCreateResponse\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"Y\n\x1bWorkflowExecutionGetRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\xb6\x01\n\tExecution\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x31\n\x04spec\x18\x02 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12:\n\x07\x63losure\x18\x03 \x01(\x0b\x32 .flyteidl.admin.ExecutionClosureR\x07\x63losure\"`\n\rExecutionList\x12\x39\n\nexecutions\x18\x01 \x03(\x0b\x32\x19.flyteidl.admin.ExecutionR\nexecutions\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"e\n\x0eLiteralMapBlob\x12\x37\n\x06values\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\x06values\x12\x12\n\x03uri\x18\x02 \x01(\tH\x00R\x03uriB\x06\n\x04\x64\x61ta\"C\n\rAbortMetadata\x12\x14\n\x05\x63\x61use\x18\x01 \x01(\tR\x05\x63\x61use\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\"\x98\x07\n\x10\x45xecutionClosure\x12>\n\x07outputs\x18\x01 \x01(\x0b\x32\x1e.flyteidl.admin.LiteralMapBlobB\x02\x18\x01H\x00R\x07outputs\x12\x35\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x1d.flyteidl.core.ExecutionErrorH\x00R\x05\x65rror\x12%\n\x0b\x61\x62ort_cause\x18\n \x01(\tB\x02\x18\x01H\x00R\nabortCause\x12\x46\n\x0e\x61\x62ort_metadata\x18\x0c \x01(\x0b\x32\x1d.flyteidl.admin.AbortMetadataH\x00R\rabortMetadata\x12@\n\x0boutput_data\x18\r \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\noutputData\x12\x46\n\x0f\x63omputed_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x0e\x63omputedInputs\x12<\n\x05phase\x18\x04 \x01(\x0e\x32&.flyteidl.core.WorkflowExecution.PhaseR\x05phase\x12\x39\n\nstarted_at\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tstartedAt\x12\x35\n\x08\x64uration\x18\x06 \x01(\x0b\x32\x19.google.protobuf.DurationR\x08\x64uration\x12\x39\n\ncreated_at\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\x12\x42\n\rnotifications\x18\t \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\x12:\n\x0bworkflow_id\x18\x0b \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nworkflowId\x12]\n\x14state_change_details\x18\x0e \x01(\x0b\x32+.flyteidl.admin.ExecutionStateChangeDetailsR\x12stateChangeDetailsB\x0f\n\routput_result\"[\n\x0eSystemMetadata\x12+\n\x11\x65xecution_cluster\x18\x01 \x01(\tR\x10\x65xecutionCluster\x12\x1c\n\tnamespace\x18\x02 \x01(\tR\tnamespace\"\x91\x05\n\x11\x45xecutionMetadata\x12\x43\n\x04mode\x18\x01 \x01(\x0e\x32/.flyteidl.admin.ExecutionMetadata.ExecutionModeR\x04mode\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\x12\x18\n\x07nesting\x18\x03 \x01(\rR\x07nesting\x12=\n\x0cscheduled_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\x0bscheduledAt\x12Z\n\x15parent_node_execution\x18\x05 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x13parentNodeExecution\x12[\n\x13reference_execution\x18\x10 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x12referenceExecution\x12G\n\x0fsystem_metadata\x18\x11 \x01(\x0b\x32\x1e.flyteidl.admin.SystemMetadataR\x0esystemMetadata\x12<\n\x0c\x61rtifact_ids\x18\x12 \x03(\x0b\x32\x19.flyteidl.core.ArtifactIDR\x0b\x61rtifactIds\"z\n\rExecutionMode\x12\n\n\x06MANUAL\x10\x00\x12\r\n\tSCHEDULED\x10\x01\x12\n\n\x06SYSTEM\x10\x02\x12\x0c\n\x08RELAUNCH\x10\x03\x12\x12\n\x0e\x43HILD_WORKFLOW\x10\x04\x12\r\n\tRECOVERED\x10\x05\x12\x0b\n\x07TRIGGER\x10\x06\"\x04\x08\x07\x10\x07J\x04\x08\x13\x10\x14\"V\n\x10NotificationList\x12\x42\n\rnotifications\x18\x01 \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\"\xd6\t\n\rExecutionSpec\x12:\n\x0blaunch_plan\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nlaunchPlan\x12\x35\n\x06inputs\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x06inputs\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\x12H\n\rnotifications\x18\x05 \x01(\x0b\x32 .flyteidl.admin.NotificationListH\x00R\rnotifications\x12!\n\x0b\x64isable_all\x18\x06 \x01(\x08H\x00R\ndisableAll\x12.\n\x06labels\x18\x07 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x08 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12I\n\x10security_context\x18\n \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12\x39\n\tauth_role\x18\x10 \x01(\x0b\x32\x18.flyteidl.admin.AuthRoleB\x02\x18\x01R\x08\x61uthRole\x12M\n\x12quality_of_service\x18\x11 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceR\x10qualityOfService\x12\'\n\x0fmax_parallelism\x18\x12 \x01(\x05R\x0emaxParallelism\x12X\n\x16raw_output_data_config\x18\x13 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12P\n\x12\x63luster_assignment\x18\x14 \x01(\x0b\x32!.flyteidl.admin.ClusterAssignmentR\x11\x63lusterAssignment\x12@\n\rinterruptible\x18\x15 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x16 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x17 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\x12\x16\n\x04tags\x18\x18 \x03(\tB\x02\x18\x01R\x04tags\x12]\n\x17\x65xecution_cluster_label\x18\x19 \x01(\x0b\x32%.flyteidl.admin.ExecutionClusterLabelR\x15\x65xecutionClusterLabel\x12\x61\n\x19\x65xecution_env_assignments\x18\x1a \x03(\x0b\x32%.flyteidl.core.ExecutionEnvAssignmentR\x17\x65xecutionEnvAssignmentsB\x18\n\x16notification_overridesJ\x04\x08\x04\x10\x05\"m\n\x19\x45xecutionTerminateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x63\x61use\x18\x02 \x01(\tR\x05\x63\x61use\"\x1c\n\x1a\x45xecutionTerminateResponse\"]\n\x1fWorkflowExecutionGetDataRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\x88\x02\n WorkflowExecutionGetDataResponse\x12\x35\n\x07outputs\x18\x01 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x07outputs\x12\x33\n\x06inputs\x18\x02 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x06inputs\x12:\n\x0b\x66ull_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\nfullInputs\x12<\n\x0c\x66ull_outputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ullOutputs\"\x8a\x01\n\x16\x45xecutionUpdateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x34\n\x05state\x18\x02 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\"\xae\x01\n\x1b\x45xecutionStateChangeDetails\x12\x34\n\x05state\x18\x01 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\x12;\n\x0boccurred_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\noccurredAt\x12\x1c\n\tprincipal\x18\x03 \x01(\tR\tprincipal\"\x19\n\x17\x45xecutionUpdateResponse\"v\n\"WorkflowExecutionGetMetricsRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x64\x65pth\x18\x02 \x01(\x05R\x05\x64\x65pth\"N\n#WorkflowExecutionGetMetricsResponse\x12\'\n\x04span\x18\x01 \x01(\x0b\x32\x13.flyteidl.core.SpanR\x04span*>\n\x0e\x45xecutionState\x12\x14\n\x10\x45XECUTION_ACTIVE\x10\x00\x12\x16\n\x12\x45XECUTION_ARCHIVED\x10\x01\x42\xba\x01\n\x12\x63om.flyteidl.adminB\x0e\x45xecutionProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1e\x66lyteidl/admin/execution.proto\x12\x0e\x66lyteidl.admin\x1a\'flyteidl/admin/cluster_assignment.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\"flyteidl/core/execution_envs.proto\x1a\x1f\x66lyteidl/core/artifact_id.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1b\x66lyteidl/core/metrics.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\x1a\'flyteidl/admin/matchable_resource.proto\"\xd6\x01\n\x16\x45xecutionCreateRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\x31\n\x04spec\x18\x04 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12\x31\n\x06inputs\x18\x05 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x06inputs\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org\"\x99\x01\n\x18\x45xecutionRelaunchRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\'\n\x0foverwrite_cache\x18\x04 \x01(\x08R\x0eoverwriteCacheJ\x04\x08\x02\x10\x03\"\xa8\x01\n\x17\x45xecutionRecoverRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\"U\n\x17\x45xecutionCreateResponse\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"Y\n\x1bWorkflowExecutionGetRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\xb6\x01\n\tExecution\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x31\n\x04spec\x18\x02 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12:\n\x07\x63losure\x18\x03 \x01(\x0b\x32 .flyteidl.admin.ExecutionClosureR\x07\x63losure\"`\n\rExecutionList\x12\x39\n\nexecutions\x18\x01 \x03(\x0b\x32\x19.flyteidl.admin.ExecutionR\nexecutions\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"e\n\x0eLiteralMapBlob\x12\x37\n\x06values\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\x06values\x12\x12\n\x03uri\x18\x02 \x01(\tH\x00R\x03uriB\x06\n\x04\x64\x61ta\"C\n\rAbortMetadata\x12\x14\n\x05\x63\x61use\x18\x01 \x01(\tR\x05\x63\x61use\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\"\x98\x07\n\x10\x45xecutionClosure\x12>\n\x07outputs\x18\x01 \x01(\x0b\x32\x1e.flyteidl.admin.LiteralMapBlobB\x02\x18\x01H\x00R\x07outputs\x12\x35\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x1d.flyteidl.core.ExecutionErrorH\x00R\x05\x65rror\x12%\n\x0b\x61\x62ort_cause\x18\n \x01(\tB\x02\x18\x01H\x00R\nabortCause\x12\x46\n\x0e\x61\x62ort_metadata\x18\x0c \x01(\x0b\x32\x1d.flyteidl.admin.AbortMetadataH\x00R\rabortMetadata\x12@\n\x0boutput_data\x18\r \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\noutputData\x12\x46\n\x0f\x63omputed_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x0e\x63omputedInputs\x12<\n\x05phase\x18\x04 \x01(\x0e\x32&.flyteidl.core.WorkflowExecution.PhaseR\x05phase\x12\x39\n\nstarted_at\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tstartedAt\x12\x35\n\x08\x64uration\x18\x06 \x01(\x0b\x32\x19.google.protobuf.DurationR\x08\x64uration\x12\x39\n\ncreated_at\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\x12\x42\n\rnotifications\x18\t \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\x12:\n\x0bworkflow_id\x18\x0b \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nworkflowId\x12]\n\x14state_change_details\x18\x0e \x01(\x0b\x32+.flyteidl.admin.ExecutionStateChangeDetailsR\x12stateChangeDetailsB\x0f\n\routput_result\"[\n\x0eSystemMetadata\x12+\n\x11\x65xecution_cluster\x18\x01 \x01(\tR\x10\x65xecutionCluster\x12\x1c\n\tnamespace\x18\x02 \x01(\tR\tnamespace\"\x91\x05\n\x11\x45xecutionMetadata\x12\x43\n\x04mode\x18\x01 \x01(\x0e\x32/.flyteidl.admin.ExecutionMetadata.ExecutionModeR\x04mode\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\x12\x18\n\x07nesting\x18\x03 \x01(\rR\x07nesting\x12=\n\x0cscheduled_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\x0bscheduledAt\x12Z\n\x15parent_node_execution\x18\x05 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x13parentNodeExecution\x12[\n\x13reference_execution\x18\x10 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x12referenceExecution\x12G\n\x0fsystem_metadata\x18\x11 \x01(\x0b\x32\x1e.flyteidl.admin.SystemMetadataR\x0esystemMetadata\x12<\n\x0c\x61rtifact_ids\x18\x12 \x03(\x0b\x32\x19.flyteidl.core.ArtifactIDR\x0b\x61rtifactIds\"z\n\rExecutionMode\x12\n\n\x06MANUAL\x10\x00\x12\r\n\tSCHEDULED\x10\x01\x12\n\n\x06SYSTEM\x10\x02\x12\x0c\n\x08RELAUNCH\x10\x03\x12\x12\n\x0e\x43HILD_WORKFLOW\x10\x04\x12\r\n\tRECOVERED\x10\x05\x12\x0b\n\x07TRIGGER\x10\x06\"\x04\x08\x07\x10\x07J\x04\x08\x13\x10\x14\"V\n\x10NotificationList\x12\x42\n\rnotifications\x18\x01 \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\"\xd6\t\n\rExecutionSpec\x12:\n\x0blaunch_plan\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nlaunchPlan\x12\x35\n\x06inputs\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x06inputs\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\x12H\n\rnotifications\x18\x05 \x01(\x0b\x32 .flyteidl.admin.NotificationListH\x00R\rnotifications\x12!\n\x0b\x64isable_all\x18\x06 \x01(\x08H\x00R\ndisableAll\x12.\n\x06labels\x18\x07 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x08 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12I\n\x10security_context\x18\n \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12\x39\n\tauth_role\x18\x10 \x01(\x0b\x32\x18.flyteidl.admin.AuthRoleB\x02\x18\x01R\x08\x61uthRole\x12M\n\x12quality_of_service\x18\x11 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceR\x10qualityOfService\x12\'\n\x0fmax_parallelism\x18\x12 \x01(\x05R\x0emaxParallelism\x12X\n\x16raw_output_data_config\x18\x13 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12P\n\x12\x63luster_assignment\x18\x14 \x01(\x0b\x32!.flyteidl.admin.ClusterAssignmentR\x11\x63lusterAssignment\x12@\n\rinterruptible\x18\x15 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x16 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x17 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\x12\x16\n\x04tags\x18\x18 \x03(\tB\x02\x18\x01R\x04tags\x12]\n\x17\x65xecution_cluster_label\x18\x19 \x01(\x0b\x32%.flyteidl.admin.ExecutionClusterLabelR\x15\x65xecutionClusterLabel\x12\x61\n\x19\x65xecution_env_assignments\x18\x1a \x03(\x0b\x32%.flyteidl.core.ExecutionEnvAssignmentR\x17\x65xecutionEnvAssignmentsB\x18\n\x16notification_overridesJ\x04\x08\x04\x10\x05\"m\n\x19\x45xecutionTerminateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x63\x61use\x18\x02 \x01(\tR\x05\x63\x61use\"\x1c\n\x1a\x45xecutionTerminateResponse\"]\n\x1fWorkflowExecutionGetDataRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\x88\x02\n WorkflowExecutionGetDataResponse\x12\x35\n\x07outputs\x18\x01 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x07outputs\x12\x33\n\x06inputs\x18\x02 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x06inputs\x12:\n\x0b\x66ull_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\nfullInputs\x12<\n\x0c\x66ull_outputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ullOutputs\"\x8a\x01\n\x16\x45xecutionUpdateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x34\n\x05state\x18\x02 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\"\xd0\x01\n\x1b\x45xecutionStateChangeDetails\x12\x34\n\x05state\x18\x01 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\x12;\n\x0boccurred_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\noccurredAt\x12\x1c\n\tprincipal\x18\x03 \x01(\tR\tprincipal\x12 \n\x0b\x64\x65scription\x18\x04 \x01(\tR\x0b\x64\x65scription\"\x19\n\x17\x45xecutionUpdateResponse\"v\n\"WorkflowExecutionGetMetricsRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x64\x65pth\x18\x02 \x01(\x05R\x05\x64\x65pth\"N\n#WorkflowExecutionGetMetricsResponse\x12\'\n\x04span\x18\x01 \x01(\x0b\x32\x13.flyteidl.core.SpanR\x04span*>\n\x0e\x45xecutionState\x12\x14\n\x10\x45XECUTION_ACTIVE\x10\x00\x12\x16\n\x12\x45XECUTION_ARCHIVED\x10\x01\x42\xba\x01\n\x12\x63om.flyteidl.adminB\x0e\x45xecutionProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -55,8 +55,8 @@ _WORKFLOWEXECUTIONGETDATARESPONSE.fields_by_name['outputs']._serialized_options = b'\030\001' _WORKFLOWEXECUTIONGETDATARESPONSE.fields_by_name['inputs']._options = None _WORKFLOWEXECUTIONGETDATARESPONSE.fields_by_name['inputs']._serialized_options = b'\030\001' - _globals['_EXECUTIONSTATE']._serialized_start=5709 - _globals['_EXECUTIONSTATE']._serialized_end=5771 + _globals['_EXECUTIONSTATE']._serialized_start=5743 + _globals['_EXECUTIONSTATE']._serialized_end=5805 _globals['_EXECUTIONCREATEREQUEST']._serialized_start=480 _globals['_EXECUTIONCREATEREQUEST']._serialized_end=694 _globals['_EXECUTIONRELAUNCHREQUEST']._serialized_start=697 @@ -98,11 +98,11 @@ _globals['_EXECUTIONUPDATEREQUEST']._serialized_start=5165 _globals['_EXECUTIONUPDATEREQUEST']._serialized_end=5303 _globals['_EXECUTIONSTATECHANGEDETAILS']._serialized_start=5306 - _globals['_EXECUTIONSTATECHANGEDETAILS']._serialized_end=5480 - _globals['_EXECUTIONUPDATERESPONSE']._serialized_start=5482 - _globals['_EXECUTIONUPDATERESPONSE']._serialized_end=5507 - _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_start=5509 - _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_end=5627 - _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_start=5629 - _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_end=5707 + _globals['_EXECUTIONSTATECHANGEDETAILS']._serialized_end=5514 + _globals['_EXECUTIONUPDATERESPONSE']._serialized_start=5516 + _globals['_EXECUTIONUPDATERESPONSE']._serialized_end=5541 + _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_start=5543 + _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_end=5661 + _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_start=5663 + _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_end=5741 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi index 86928f23df..cf1da03897 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi @@ -271,14 +271,16 @@ class ExecutionUpdateRequest(_message.Message): def __init__(self, id: _Optional[_Union[_identifier_pb2.WorkflowExecutionIdentifier, _Mapping]] = ..., state: _Optional[_Union[ExecutionState, str]] = ...) -> None: ... class ExecutionStateChangeDetails(_message.Message): - __slots__ = ["state", "occurred_at", "principal"] + __slots__ = ["state", "occurred_at", "principal", "description"] STATE_FIELD_NUMBER: _ClassVar[int] OCCURRED_AT_FIELD_NUMBER: _ClassVar[int] PRINCIPAL_FIELD_NUMBER: _ClassVar[int] + DESCRIPTION_FIELD_NUMBER: _ClassVar[int] state: ExecutionState occurred_at: _timestamp_pb2.Timestamp principal: str - def __init__(self, state: _Optional[_Union[ExecutionState, str]] = ..., occurred_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., principal: _Optional[str] = ...) -> None: ... + description: str + def __init__(self, state: _Optional[_Union[ExecutionState, str]] = ..., occurred_at: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., principal: _Optional[str] = ..., description: _Optional[str] = ...) -> None: ... class ExecutionUpdateResponse(_message.Message): __slots__ = [] diff --git a/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.py b/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.py index 42af3da7a2..118973c0d3 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.py @@ -13,7 +13,7 @@ -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1d\x66lyteidl/admin/schedule.proto\x12\x0e\x66lyteidl.admin\"T\n\tFixedRate\x12\x14\n\x05value\x18\x01 \x01(\rR\x05value\x12\x31\n\x04unit\x18\x02 \x01(\x0e\x32\x1d.flyteidl.admin.FixedRateUnitR\x04unit\"B\n\x0c\x43ronSchedule\x12\x1a\n\x08schedule\x18\x01 \x01(\tR\x08schedule\x12\x16\n\x06offset\x18\x02 \x01(\tR\x06offset\"\xfa\x01\n\x08Schedule\x12-\n\x0f\x63ron_expression\x18\x01 \x01(\tB\x02\x18\x01H\x00R\x0e\x63ronExpression\x12/\n\x04rate\x18\x02 \x01(\x0b\x32\x19.flyteidl.admin.FixedRateH\x00R\x04rate\x12\x43\n\rcron_schedule\x18\x04 \x01(\x0b\x32\x1c.flyteidl.admin.CronScheduleH\x00R\x0c\x63ronSchedule\x12\x33\n\x16kickoff_time_input_arg\x18\x03 \x01(\tR\x13kickoffTimeInputArgB\x14\n\x12ScheduleExpression*.\n\rFixedRateUnit\x12\n\n\x06MINUTE\x10\x00\x12\x08\n\x04HOUR\x10\x01\x12\x07\n\x03\x44\x41Y\x10\x02\x42\xb9\x01\n\x12\x63om.flyteidl.adminB\rScheduleProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1d\x66lyteidl/admin/schedule.proto\x12\x0e\x66lyteidl.admin\"T\n\tFixedRate\x12\x14\n\x05value\x18\x01 \x01(\rR\x05value\x12\x31\n\x04unit\x18\x02 \x01(\x0e\x32\x1d.flyteidl.admin.FixedRateUnitR\x04unit\"B\n\x0c\x43ronSchedule\x12\x1a\n\x08schedule\x18\x01 \x01(\tR\x08schedule\x12\x16\n\x06offset\x18\x02 \x01(\tR\x06offset\"\xc6\x02\n\x08Schedule\x12-\n\x0f\x63ron_expression\x18\x01 \x01(\tB\x02\x18\x01H\x00R\x0e\x63ronExpression\x12/\n\x04rate\x18\x02 \x01(\x0b\x32\x19.flyteidl.admin.FixedRateH\x00R\x04rate\x12\x43\n\rcron_schedule\x18\x04 \x01(\x0b\x32\x1c.flyteidl.admin.CronScheduleH\x00R\x0c\x63ronSchedule\x12\x33\n\x16kickoff_time_input_arg\x18\x03 \x01(\tR\x13kickoffTimeInputArg\x12J\n\x10scheduler_policy\x18\x05 \x01(\x0b\x32\x1f.flyteidl.admin.SchedulerPolicyR\x0fschedulerPolicyB\x14\n\x12ScheduleExpression\"^\n\x0fSchedulerPolicy\x12\x10\n\x03max\x18\x01 \x01(\rR\x03max\x12\x39\n\x06policy\x18\x02 \x01(\x0e\x32!.flyteidl.admin.ConcurrencyPolicyR\x06policy*.\n\rFixedRateUnit\x12\n\n\x06MINUTE\x10\x00\x12\x08\n\x04HOUR\x10\x01\x12\x07\n\x03\x44\x41Y\x10\x02*F\n\x11\x43oncurrencyPolicy\x12\x0f\n\x0bUNSPECIFIED\x10\x00\x12\x08\n\x04WAIT\x10\x01\x12\t\n\x05\x41\x42ORT\x10\x02\x12\x0b\n\x07REPLACE\x10\x03*<\n\x10\x43oncurrencyLevel\x12\x0f\n\x0bLAUNCH_PLAN\x10\x00\x12\x17\n\x13LAUNCH_PLAN_VERSION\x10\x01\x42\xb9\x01\n\x12\x63om.flyteidl.adminB\rScheduleProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -24,12 +24,18 @@ DESCRIPTOR._serialized_options = b'\n\022com.flyteidl.adminB\rScheduleProtoP\001Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\242\002\003FAX\252\002\016Flyteidl.Admin\312\002\016Flyteidl\\Admin\342\002\032Flyteidl\\Admin\\GPBMetadata\352\002\017Flyteidl::Admin' _SCHEDULE.fields_by_name['cron_expression']._options = None _SCHEDULE.fields_by_name['cron_expression']._serialized_options = b'\030\001' - _globals['_FIXEDRATEUNIT']._serialized_start=456 - _globals['_FIXEDRATEUNIT']._serialized_end=502 + _globals['_FIXEDRATEUNIT']._serialized_start=628 + _globals['_FIXEDRATEUNIT']._serialized_end=674 + _globals['_CONCURRENCYPOLICY']._serialized_start=676 + _globals['_CONCURRENCYPOLICY']._serialized_end=746 + _globals['_CONCURRENCYLEVEL']._serialized_start=748 + _globals['_CONCURRENCYLEVEL']._serialized_end=808 _globals['_FIXEDRATE']._serialized_start=49 _globals['_FIXEDRATE']._serialized_end=133 _globals['_CRONSCHEDULE']._serialized_start=135 _globals['_CRONSCHEDULE']._serialized_end=201 _globals['_SCHEDULE']._serialized_start=204 - _globals['_SCHEDULE']._serialized_end=454 + _globals['_SCHEDULE']._serialized_end=530 + _globals['_SCHEDULERPOLICY']._serialized_start=532 + _globals['_SCHEDULERPOLICY']._serialized_end=626 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.pyi index a9f1a2d133..bb7b84943d 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/admin/schedule_pb2.pyi @@ -10,9 +10,27 @@ class FixedRateUnit(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): MINUTE: _ClassVar[FixedRateUnit] HOUR: _ClassVar[FixedRateUnit] DAY: _ClassVar[FixedRateUnit] + +class ConcurrencyPolicy(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): + __slots__ = [] + UNSPECIFIED: _ClassVar[ConcurrencyPolicy] + WAIT: _ClassVar[ConcurrencyPolicy] + ABORT: _ClassVar[ConcurrencyPolicy] + REPLACE: _ClassVar[ConcurrencyPolicy] + +class ConcurrencyLevel(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): + __slots__ = [] + LAUNCH_PLAN: _ClassVar[ConcurrencyLevel] + LAUNCH_PLAN_VERSION: _ClassVar[ConcurrencyLevel] MINUTE: FixedRateUnit HOUR: FixedRateUnit DAY: FixedRateUnit +UNSPECIFIED: ConcurrencyPolicy +WAIT: ConcurrencyPolicy +ABORT: ConcurrencyPolicy +REPLACE: ConcurrencyPolicy +LAUNCH_PLAN: ConcurrencyLevel +LAUNCH_PLAN_VERSION: ConcurrencyLevel class FixedRate(_message.Message): __slots__ = ["value", "unit"] @@ -31,13 +49,23 @@ class CronSchedule(_message.Message): def __init__(self, schedule: _Optional[str] = ..., offset: _Optional[str] = ...) -> None: ... class Schedule(_message.Message): - __slots__ = ["cron_expression", "rate", "cron_schedule", "kickoff_time_input_arg"] + __slots__ = ["cron_expression", "rate", "cron_schedule", "kickoff_time_input_arg", "scheduler_policy"] CRON_EXPRESSION_FIELD_NUMBER: _ClassVar[int] RATE_FIELD_NUMBER: _ClassVar[int] CRON_SCHEDULE_FIELD_NUMBER: _ClassVar[int] KICKOFF_TIME_INPUT_ARG_FIELD_NUMBER: _ClassVar[int] + SCHEDULER_POLICY_FIELD_NUMBER: _ClassVar[int] cron_expression: str rate: FixedRate cron_schedule: CronSchedule kickoff_time_input_arg: str - def __init__(self, cron_expression: _Optional[str] = ..., rate: _Optional[_Union[FixedRate, _Mapping]] = ..., cron_schedule: _Optional[_Union[CronSchedule, _Mapping]] = ..., kickoff_time_input_arg: _Optional[str] = ...) -> None: ... + scheduler_policy: SchedulerPolicy + def __init__(self, cron_expression: _Optional[str] = ..., rate: _Optional[_Union[FixedRate, _Mapping]] = ..., cron_schedule: _Optional[_Union[CronSchedule, _Mapping]] = ..., kickoff_time_input_arg: _Optional[str] = ..., scheduler_policy: _Optional[_Union[SchedulerPolicy, _Mapping]] = ...) -> None: ... + +class SchedulerPolicy(_message.Message): + __slots__ = ["max", "policy"] + MAX_FIELD_NUMBER: _ClassVar[int] + POLICY_FIELD_NUMBER: _ClassVar[int] + max: int + policy: ConcurrencyPolicy + def __init__(self, max: _Optional[int] = ..., policy: _Optional[_Union[ConcurrencyPolicy, str]] = ...) -> None: ... diff --git a/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.py b/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.py index 2ff8f47010..b168afeff9 100644 --- a/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.py @@ -15,7 +15,7 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1d\x66lyteidl/core/execution.proto\x12\rflyteidl.core\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xa7\x01\n\x11WorkflowExecution\"\x91\x01\n\x05Phase\x12\r\n\tUNDEFINED\x10\x00\x12\n\n\x06QUEUED\x10\x01\x12\x0b\n\x07RUNNING\x10\x02\x12\x0e\n\nSUCCEEDING\x10\x03\x12\r\n\tSUCCEEDED\x10\x04\x12\x0b\n\x07\x46\x41ILING\x10\x05\x12\n\n\x06\x46\x41ILED\x10\x06\x12\x0b\n\x07\x41\x42ORTED\x10\x07\x12\r\n\tTIMED_OUT\x10\x08\x12\x0c\n\x08\x41\x42ORTING\x10\t\"\xb6\x01\n\rNodeExecution\"\xa4\x01\n\x05Phase\x12\r\n\tUNDEFINED\x10\x00\x12\n\n\x06QUEUED\x10\x01\x12\x0b\n\x07RUNNING\x10\x02\x12\r\n\tSUCCEEDED\x10\x03\x12\x0b\n\x07\x46\x41ILING\x10\x04\x12\n\n\x06\x46\x41ILED\x10\x05\x12\x0b\n\x07\x41\x42ORTED\x10\x06\x12\x0b\n\x07SKIPPED\x10\x07\x12\r\n\tTIMED_OUT\x10\x08\x12\x13\n\x0f\x44YNAMIC_RUNNING\x10\t\x12\r\n\tRECOVERED\x10\n\"\x96\x01\n\rTaskExecution\"\x84\x01\n\x05Phase\x12\r\n\tUNDEFINED\x10\x00\x12\n\n\x06QUEUED\x10\x01\x12\x0b\n\x07RUNNING\x10\x02\x12\r\n\tSUCCEEDED\x10\x03\x12\x0b\n\x07\x41\x42ORTED\x10\x04\x12\n\n\x06\x46\x41ILED\x10\x05\x12\x10\n\x0cINITIALIZING\x10\x06\x12\x19\n\x15WAITING_FOR_RESOURCES\x10\x07\"\x9a\x02\n\x0e\x45xecutionError\x12\x12\n\x04\x63ode\x18\x01 \x01(\tR\x04\x63ode\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12\x1b\n\terror_uri\x18\x03 \x01(\tR\x08\x65rrorUri\x12;\n\x04kind\x18\x04 \x01(\x0e\x32\'.flyteidl.core.ExecutionError.ErrorKindR\x04kind\x12\x38\n\ttimestamp\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x16\n\x06worker\x18\x06 \x01(\tR\x06worker\".\n\tErrorKind\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04USER\x10\x01\x12\n\n\x06SYSTEM\x10\x02\"\xb2\x02\n\x07TaskLog\x12\x10\n\x03uri\x18\x01 \x01(\tR\x03uri\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12K\n\x0emessage_format\x18\x03 \x01(\x0e\x32$.flyteidl.core.TaskLog.MessageFormatR\rmessageFormat\x12+\n\x03ttl\x18\x04 \x01(\x0b\x32\x19.google.protobuf.DurationR\x03ttl\x12*\n\x10ShowWhilePending\x18\x05 \x01(\x08R\x10ShowWhilePending\x12*\n\x10HideOnceFinished\x18\x06 \x01(\x08R\x10HideOnceFinished\"/\n\rMessageFormat\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x07\n\x03\x43SV\x10\x01\x12\x08\n\x04JSON\x10\x02\"Z\n\x14QualityOfServiceSpec\x12\x42\n\x0fqueueing_budget\x18\x01 \x01(\x0b\x32\x19.google.protobuf.DurationR\x0equeueingBudget\"\xce\x01\n\x10QualityOfService\x12:\n\x04tier\x18\x01 \x01(\x0e\x32$.flyteidl.core.QualityOfService.TierH\x00R\x04tier\x12\x39\n\x04spec\x18\x02 \x01(\x0b\x32#.flyteidl.core.QualityOfServiceSpecH\x00R\x04spec\"4\n\x04Tier\x12\r\n\tUNDEFINED\x10\x00\x12\x08\n\x04HIGH\x10\x01\x12\n\n\x06MEDIUM\x10\x02\x12\x07\n\x03LOW\x10\x03\x42\r\n\x0b\x64\x65signationB\xb4\x01\n\x11\x63om.flyteidl.coreB\x0e\x45xecutionProtoP\x01Z:github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core\xa2\x02\x03\x46\x43X\xaa\x02\rFlyteidl.Core\xca\x02\rFlyteidl\\Core\xe2\x02\x19\x46lyteidl\\Core\\GPBMetadata\xea\x02\x0e\x46lyteidl::Coreb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1d\x66lyteidl/core/execution.proto\x12\rflyteidl.core\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xb4\x01\n\x11WorkflowExecution\"\x9e\x01\n\x05Phase\x12\r\n\tUNDEFINED\x10\x00\x12\n\n\x06QUEUED\x10\x01\x12\x0b\n\x07RUNNING\x10\x02\x12\x0e\n\nSUCCEEDING\x10\x03\x12\r\n\tSUCCEEDED\x10\x04\x12\x0b\n\x07\x46\x41ILING\x10\x05\x12\n\n\x06\x46\x41ILED\x10\x06\x12\x0b\n\x07\x41\x42ORTED\x10\x07\x12\r\n\tTIMED_OUT\x10\x08\x12\x0c\n\x08\x41\x42ORTING\x10\t\x12\x0b\n\x07PENDING\x10\n\"\xb6\x01\n\rNodeExecution\"\xa4\x01\n\x05Phase\x12\r\n\tUNDEFINED\x10\x00\x12\n\n\x06QUEUED\x10\x01\x12\x0b\n\x07RUNNING\x10\x02\x12\r\n\tSUCCEEDED\x10\x03\x12\x0b\n\x07\x46\x41ILING\x10\x04\x12\n\n\x06\x46\x41ILED\x10\x05\x12\x0b\n\x07\x41\x42ORTED\x10\x06\x12\x0b\n\x07SKIPPED\x10\x07\x12\r\n\tTIMED_OUT\x10\x08\x12\x13\n\x0f\x44YNAMIC_RUNNING\x10\t\x12\r\n\tRECOVERED\x10\n\"\x96\x01\n\rTaskExecution\"\x84\x01\n\x05Phase\x12\r\n\tUNDEFINED\x10\x00\x12\n\n\x06QUEUED\x10\x01\x12\x0b\n\x07RUNNING\x10\x02\x12\r\n\tSUCCEEDED\x10\x03\x12\x0b\n\x07\x41\x42ORTED\x10\x04\x12\n\n\x06\x46\x41ILED\x10\x05\x12\x10\n\x0cINITIALIZING\x10\x06\x12\x19\n\x15WAITING_FOR_RESOURCES\x10\x07\"\x9a\x02\n\x0e\x45xecutionError\x12\x12\n\x04\x63ode\x18\x01 \x01(\tR\x04\x63ode\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12\x1b\n\terror_uri\x18\x03 \x01(\tR\x08\x65rrorUri\x12;\n\x04kind\x18\x04 \x01(\x0e\x32\'.flyteidl.core.ExecutionError.ErrorKindR\x04kind\x12\x38\n\ttimestamp\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x16\n\x06worker\x18\x06 \x01(\tR\x06worker\".\n\tErrorKind\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04USER\x10\x01\x12\n\n\x06SYSTEM\x10\x02\"\xb2\x02\n\x07TaskLog\x12\x10\n\x03uri\x18\x01 \x01(\tR\x03uri\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12K\n\x0emessage_format\x18\x03 \x01(\x0e\x32$.flyteidl.core.TaskLog.MessageFormatR\rmessageFormat\x12+\n\x03ttl\x18\x04 \x01(\x0b\x32\x19.google.protobuf.DurationR\x03ttl\x12*\n\x10ShowWhilePending\x18\x05 \x01(\x08R\x10ShowWhilePending\x12*\n\x10HideOnceFinished\x18\x06 \x01(\x08R\x10HideOnceFinished\"/\n\rMessageFormat\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x07\n\x03\x43SV\x10\x01\x12\x08\n\x04JSON\x10\x02\"Z\n\x14QualityOfServiceSpec\x12\x42\n\x0fqueueing_budget\x18\x01 \x01(\x0b\x32\x19.google.protobuf.DurationR\x0equeueingBudget\"\xce\x01\n\x10QualityOfService\x12:\n\x04tier\x18\x01 \x01(\x0e\x32$.flyteidl.core.QualityOfService.TierH\x00R\x04tier\x12\x39\n\x04spec\x18\x02 \x01(\x0b\x32#.flyteidl.core.QualityOfServiceSpecH\x00R\x04spec\"4\n\x04Tier\x12\r\n\tUNDEFINED\x10\x00\x12\x08\n\x04HIGH\x10\x01\x12\n\n\x06MEDIUM\x10\x02\x12\x07\n\x03LOW\x10\x03\x42\r\n\x0b\x64\x65signationB\xb4\x01\n\x11\x63om.flyteidl.coreB\x0e\x45xecutionProtoP\x01Z:github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core\xa2\x02\x03\x46\x43X\xaa\x02\rFlyteidl.Core\xca\x02\rFlyteidl\\Core\xe2\x02\x19\x46lyteidl\\Core\\GPBMetadata\xea\x02\x0e\x46lyteidl::Coreb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -25,29 +25,29 @@ DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b'\n\021com.flyteidl.coreB\016ExecutionProtoP\001Z:github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core\242\002\003FCX\252\002\rFlyteidl.Core\312\002\rFlyteidl\\Core\342\002\031Flyteidl\\Core\\GPBMetadata\352\002\016Flyteidl::Core' _globals['_WORKFLOWEXECUTION']._serialized_start=114 - _globals['_WORKFLOWEXECUTION']._serialized_end=281 + _globals['_WORKFLOWEXECUTION']._serialized_end=294 _globals['_WORKFLOWEXECUTION_PHASE']._serialized_start=136 - _globals['_WORKFLOWEXECUTION_PHASE']._serialized_end=281 - _globals['_NODEEXECUTION']._serialized_start=284 - _globals['_NODEEXECUTION']._serialized_end=466 - _globals['_NODEEXECUTION_PHASE']._serialized_start=302 - _globals['_NODEEXECUTION_PHASE']._serialized_end=466 - _globals['_TASKEXECUTION']._serialized_start=469 - _globals['_TASKEXECUTION']._serialized_end=619 - _globals['_TASKEXECUTION_PHASE']._serialized_start=487 - _globals['_TASKEXECUTION_PHASE']._serialized_end=619 - _globals['_EXECUTIONERROR']._serialized_start=622 - _globals['_EXECUTIONERROR']._serialized_end=904 - _globals['_EXECUTIONERROR_ERRORKIND']._serialized_start=858 - _globals['_EXECUTIONERROR_ERRORKIND']._serialized_end=904 - _globals['_TASKLOG']._serialized_start=907 - _globals['_TASKLOG']._serialized_end=1213 - _globals['_TASKLOG_MESSAGEFORMAT']._serialized_start=1166 - _globals['_TASKLOG_MESSAGEFORMAT']._serialized_end=1213 - _globals['_QUALITYOFSERVICESPEC']._serialized_start=1215 - _globals['_QUALITYOFSERVICESPEC']._serialized_end=1305 - _globals['_QUALITYOFSERVICE']._serialized_start=1308 - _globals['_QUALITYOFSERVICE']._serialized_end=1514 - _globals['_QUALITYOFSERVICE_TIER']._serialized_start=1447 - _globals['_QUALITYOFSERVICE_TIER']._serialized_end=1499 + _globals['_WORKFLOWEXECUTION_PHASE']._serialized_end=294 + _globals['_NODEEXECUTION']._serialized_start=297 + _globals['_NODEEXECUTION']._serialized_end=479 + _globals['_NODEEXECUTION_PHASE']._serialized_start=315 + _globals['_NODEEXECUTION_PHASE']._serialized_end=479 + _globals['_TASKEXECUTION']._serialized_start=482 + _globals['_TASKEXECUTION']._serialized_end=632 + _globals['_TASKEXECUTION_PHASE']._serialized_start=500 + _globals['_TASKEXECUTION_PHASE']._serialized_end=632 + _globals['_EXECUTIONERROR']._serialized_start=635 + _globals['_EXECUTIONERROR']._serialized_end=917 + _globals['_EXECUTIONERROR_ERRORKIND']._serialized_start=871 + _globals['_EXECUTIONERROR_ERRORKIND']._serialized_end=917 + _globals['_TASKLOG']._serialized_start=920 + _globals['_TASKLOG']._serialized_end=1226 + _globals['_TASKLOG_MESSAGEFORMAT']._serialized_start=1179 + _globals['_TASKLOG_MESSAGEFORMAT']._serialized_end=1226 + _globals['_QUALITYOFSERVICESPEC']._serialized_start=1228 + _globals['_QUALITYOFSERVICESPEC']._serialized_end=1318 + _globals['_QUALITYOFSERVICE']._serialized_start=1321 + _globals['_QUALITYOFSERVICE']._serialized_end=1527 + _globals['_QUALITYOFSERVICE_TIER']._serialized_start=1460 + _globals['_QUALITYOFSERVICE_TIER']._serialized_end=1512 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.pyi index 08f1937c08..4ba31cf43c 100644 --- a/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/core/execution_pb2.pyi @@ -21,6 +21,7 @@ class WorkflowExecution(_message.Message): ABORTED: _ClassVar[WorkflowExecution.Phase] TIMED_OUT: _ClassVar[WorkflowExecution.Phase] ABORTING: _ClassVar[WorkflowExecution.Phase] + PENDING: _ClassVar[WorkflowExecution.Phase] UNDEFINED: WorkflowExecution.Phase QUEUED: WorkflowExecution.Phase RUNNING: WorkflowExecution.Phase @@ -31,6 +32,7 @@ class WorkflowExecution(_message.Message): ABORTED: WorkflowExecution.Phase TIMED_OUT: WorkflowExecution.Phase ABORTING: WorkflowExecution.Phase + PENDING: WorkflowExecution.Phase def __init__(self) -> None: ... class NodeExecution(_message.Message): diff --git a/flyteidl/gen/pb_rust/flyteidl.admin.rs b/flyteidl/gen/pb_rust/flyteidl.admin.rs index 3dc3bf20f2..177dd0ffff 100644 --- a/flyteidl/gen/pb_rust/flyteidl.admin.rs +++ b/flyteidl/gen/pb_rust/flyteidl.admin.rs @@ -1821,6 +1821,9 @@ pub struct ExecutionStateChangeDetails { /// Identifies the entity (if any) responsible for causing the state change of the execution #[prost(string, tag="3")] pub principal: ::prost::alloc::string::String, + /// Includes the reason for the `PENDING` phase + #[prost(string, tag="4")] + pub description: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, Copy, PartialEq, ::prost::Message)] @@ -1905,6 +1908,8 @@ pub struct Schedule { /// Name of the input variable that the kickoff time will be supplied to when the workflow is kicked off. #[prost(string, tag="3")] pub kickoff_time_input_arg: ::prost::alloc::string::String, + #[prost(message, optional, tag="5")] + pub scheduler_policy: ::core::option::Option, #[prost(oneof="schedule::ScheduleExpression", tags="1, 2, 4")] pub schedule_expression: ::core::option::Option, } @@ -1923,6 +1928,16 @@ pub mod schedule { CronSchedule(super::CronSchedule), } } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] +pub struct SchedulerPolicy { + /// Defines how many executions with this launch plan can run in parallel + #[prost(uint32, tag="1")] + pub max: u32, + /// Defines how to handle the execution when the max concurrency is reached. + #[prost(enumeration="ConcurrencyPolicy", tag="2")] + pub policy: i32, +} /// Represents a frequency at which to run a schedule. #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] @@ -1953,6 +1968,69 @@ impl FixedRateUnit { } } } +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ConcurrencyPolicy { + Unspecified = 0, + /// wait for previous executions to terminate before starting a new one + Wait = 1, + /// fail the CreateExecution request and do not permit the execution to start + Abort = 2, + /// terminate the oldest execution when the concurrency limit is reached and immediately begin proceeding with the new execution + Replace = 3, +} +impl ConcurrencyPolicy { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ConcurrencyPolicy::Unspecified => "UNSPECIFIED", + ConcurrencyPolicy::Wait => "WAIT", + ConcurrencyPolicy::Abort => "ABORT", + ConcurrencyPolicy::Replace => "REPLACE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "UNSPECIFIED" => Some(Self::Unspecified), + "WAIT" => Some(Self::Wait), + "ABORT" => Some(Self::Abort), + "REPLACE" => Some(Self::Replace), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ConcurrencyLevel { + /// Applies concurrency limits across all launch plan versions. + LaunchPlan = 0, + /// Applies concurrency at the versioned launch plan level + LaunchPlanVersion = 1, +} +impl ConcurrencyLevel { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ConcurrencyLevel::LaunchPlan => "LAUNCH_PLAN", + ConcurrencyLevel::LaunchPlanVersion => "LAUNCH_PLAN_VERSION", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "LAUNCH_PLAN" => Some(Self::LaunchPlan), + "LAUNCH_PLAN_VERSION" => Some(Self::LaunchPlanVersion), + _ => None, + } + } +} /// Request to register a launch plan. The included LaunchPlanSpec may have a complete or incomplete set of inputs required /// to launch a workflow execution. By default all launch plans are registered in state INACTIVE. If you wish to /// set the state to ACTIVE, you must submit a LaunchPlanUpdateRequest, after you have successfully created a launch plan. diff --git a/flyteidl/gen/pb_rust/flyteidl.core.rs b/flyteidl/gen/pb_rust/flyteidl.core.rs index 6c5a5c667f..7b52353a86 100644 --- a/flyteidl/gen/pb_rust/flyteidl.core.rs +++ b/flyteidl/gen/pb_rust/flyteidl.core.rs @@ -2001,6 +2001,7 @@ pub mod workflow_execution { Aborted = 7, TimedOut = 8, Aborting = 9, + Pending = 10, } impl Phase { /// String value of the enum field names used in the ProtoBuf definition. @@ -2019,6 +2020,7 @@ pub mod workflow_execution { Phase::Aborted => "ABORTED", Phase::TimedOut => "TIMED_OUT", Phase::Aborting => "ABORTING", + Phase::Pending => "PENDING", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2034,6 +2036,7 @@ pub mod workflow_execution { "ABORTED" => Some(Self::Aborted), "TIMED_OUT" => Some(Self::TimedOut), "ABORTING" => Some(Self::Aborting), + "PENDING" => Some(Self::Pending), _ => None, } } diff --git a/flyteidl/protos/flyteidl/admin/execution.proto b/flyteidl/protos/flyteidl/admin/execution.proto index 2c629cce19..5f904e3e36 100644 --- a/flyteidl/protos/flyteidl/admin/execution.proto +++ b/flyteidl/protos/flyteidl/admin/execution.proto @@ -77,7 +77,7 @@ message ExecutionRecoverRequest { string name = 2; // Additional metadata which will be used to overwrite any metadata in the reference execution when triggering a recovery execution. - ExecutionMetadata metadata = 3; + ExecutionMetadata metadata = 3; } // The unique identifier for a successfully created execution. @@ -411,6 +411,9 @@ message ExecutionStateChangeDetails { // Identifies the entity (if any) responsible for causing the state change of the execution string principal = 3; + + // Includes the reason for the `PENDING` phase + string description = 4; } message ExecutionUpdateResponse {} diff --git a/flyteidl/protos/flyteidl/admin/schedule.proto b/flyteidl/protos/flyteidl/admin/schedule.proto index 6bcbd90140..e0ec001bbf 100644 --- a/flyteidl/protos/flyteidl/admin/schedule.proto +++ b/flyteidl/protos/flyteidl/admin/schedule.proto @@ -40,4 +40,35 @@ message Schedule { // Name of the input variable that the kickoff time will be supplied to when the workflow is kicked off. string kickoff_time_input_arg = 3; + + SchedulerPolicy scheduler_policy = 5; +} + +message SchedulerPolicy { + // Defines how many executions with this launch plan can run in parallel + uint32 max = 1; + + // Defines how to handle the execution when the max concurrency is reached. + ConcurrencyPolicy policy = 2; +} + +enum ConcurrencyPolicy { + UNSPECIFIED = 0; + + // wait for previous executions to terminate before starting a new one + WAIT = 1; + + // fail the CreateExecution request and do not permit the execution to start + ABORT = 2; + + // terminate the oldest execution when the concurrency limit is reached and immediately begin proceeding with the new execution + REPLACE = 3; +} + +enum ConcurrencyLevel { + // Applies concurrency limits across all launch plan versions. + LAUNCH_PLAN = 0; + + // Applies concurrency at the versioned launch plan level + LAUNCH_PLAN_VERSION = 1; } diff --git a/flyteidl/protos/flyteidl/core/execution.proto b/flyteidl/protos/flyteidl/core/execution.proto index 3b9bfbbbb7..1e35cd7aa7 100644 --- a/flyteidl/protos/flyteidl/core/execution.proto +++ b/flyteidl/protos/flyteidl/core/execution.proto @@ -20,6 +20,7 @@ message WorkflowExecution { ABORTED = 7; TIMED_OUT = 8; ABORTING = 9; + PENDING = 10; } }