From c6fc1cdc634693a3f6143b8a9150f6ddc716f152 Mon Sep 17 00:00:00 2001 From: jaredoconnell Date: Fri, 4 Aug 2023 10:16:47 -0400 Subject: [PATCH 01/54] Added workflow pre-validation Also improved debug messages --- cmd/run-plugin/run.go | 17 +- internal/step/dummy/provider.go | 4 +- internal/step/dummy/provider_test.go | 2 +- internal/step/foreach/provider.go | 12 +- internal/step/lifecycle.go | 2 + internal/step/plugin/provider.go | 140 ++++++++++----- internal/step/plugin/provider_test.go | 8 +- internal/step/provider.go | 1 + workflow/any.go | 9 + workflow/executor.go | 140 ++++++++++++++- workflow/executor_test.go | 189 +++++++++++++++++--- workflow/workflow.go | 2 +- workflow/workflow_test.go | 246 ++++++++++++++++++-------- 13 files changed, 618 insertions(+), 154 deletions(-) diff --git a/cmd/run-plugin/run.go b/cmd/run-plugin/run.go index be241dee..32d230ca 100644 --- a/cmd/run-plugin/run.go +++ b/cmd/run-plugin/run.go @@ -6,6 +6,8 @@ import ( "flag" "fmt" "go.flow.arcalot.io/deployer" + "go.flow.arcalot.io/pluginsdk/plugin" + "go.flow.arcalot.io/pluginsdk/schema" podman "go.flow.arcalot.io/podmandeployer" "os" "os/signal" @@ -74,12 +76,19 @@ func main() { ctrlC := make(chan os.Signal, 1) signal.Notify(ctrlC, os.Interrupt) + // Set up the signal channel to send cancel signal on ctrl-c + toStepSignals := make(chan schema.Input, 3) ctx, cancel := context.WithCancel(context.Background()) defer cancel() go func() { select { case <-ctrlC: - fmt.Println("Received CTRL-C. Cancelling the context to cancel the step...") + fmt.Println("Received CTRL-C. Sending cancel signal...") + toStepSignals <- schema.Input{ + ID: plugin.CancellationSignalSchema.ID(), + InputData: make(map[string]any), + } + fmt.Println("Signal sent.") cancel() case <-ctx.Done(): // Done here. @@ -120,7 +129,11 @@ func main() { panic(err) } fmt.Printf("Running step %s\n", stepID) - outputID, outputData, err := atpClient.Execute(stepID, input) + outputID, outputData, err := atpClient.Execute( + schema.Input{ID: stepID, InputData: input}, + nil, + nil, + ) output := map[string]any{ "outputID": outputID, "outputData": outputData, diff --git a/internal/step/dummy/provider.go b/internal/step/dummy/provider.go index cc99f333..c1b06f40 100644 --- a/internal/step/dummy/provider.go +++ b/internal/step/dummy/provider.go @@ -176,7 +176,7 @@ func (r *runnableStep) Lifecycle(_ map[string]any) (result step.Lifecycle[step.L }, nil } -func (r *runnableStep) Start(_ map[string]any, stageChangeHandler step.StageChangeHandler) (step.RunningStep, error) { +func (r *runnableStep) Start(_ map[string]any, runID string, stageChangeHandler step.StageChangeHandler) (step.RunningStep, error) { ctx, cancel := context.WithCancel(context.Background()) @@ -189,6 +189,7 @@ func (r *runnableStep) Start(_ map[string]any, stageChangeHandler step.StageChan // the ProvideInputStage is not blocked. name: make(chan string, 1), state: step.RunningStepStateStarting, + runID: runID, } go s.run() @@ -204,6 +205,7 @@ type runningStep struct { name chan string state step.RunningStepState inputAvailable bool + runID string } func (r *runningStep) State() step.RunningStepState { diff --git a/internal/step/dummy/provider_test.go b/internal/step/dummy/provider_test.go index 6ac333c2..d43fe4ff 100644 --- a/internal/step/dummy/provider_test.go +++ b/internal/step/dummy/provider_test.go @@ -49,7 +49,7 @@ func TestProvider(t *testing.T) { message: make(chan string), } - running, err := runnable.Start(map[string]any{}, handler) + running, err := runnable.Start(map[string]any{}, t.Name(), handler) assert.NoError(t, err) assert.NoError(t, running.ProvideStageInput("greet", map[string]any{ "name": "Arca Lot", diff --git a/internal/step/foreach/provider.go b/internal/step/foreach/provider.go index b0fa5d37..a8defac3 100644 --- a/internal/step/foreach/provider.go +++ b/internal/step/foreach/provider.go @@ -319,9 +319,10 @@ func (r *runnableStep) RunSchema() map[string]*schema.PropertySchema { return map[string]*schema.PropertySchema{} } -func (r *runnableStep) Start(_ map[string]any, stageChangeHandler step.StageChangeHandler) (step.RunningStep, error) { +func (r *runnableStep) Start(_ map[string]any, runID string, stageChangeHandler step.StageChangeHandler) (step.RunningStep, error) { ctx, cancel := context.WithCancel(context.Background()) rs := &runningStep{ + runID: runID, ctx: ctx, cancel: cancel, lock: &sync.Mutex{}, @@ -338,6 +339,7 @@ func (r *runnableStep) Start(_ map[string]any, stageChangeHandler step.StageChan } type runningStep struct { + runID string workflow workflow.ExecutableWorkflow currentStage StageID lock *sync.Mutex @@ -363,13 +365,13 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro _, err := r.workflow.Input().Unserialize(item) if err != nil { r.lock.Unlock() - return fmt.Errorf("invalid input item %d for subworkflow (%w)", i, err) + return fmt.Errorf("invalid input item %d for subworkflow (%w) for run/step %s", i, err, r.runID) } input[i] = item } if r.inputAvailable { r.lock.Unlock() - return fmt.Errorf("input for execute workflow provided twice") + return fmt.Errorf("input for execute workflow provided twice for run/step %s", r.runID) } if r.currentState == step.RunningStepStateWaitingForInput && r.currentStage == StageIDExecute { r.currentState = step.RunningStepStateRunning @@ -435,7 +437,7 @@ func (r *runningStep) run() { itemOutputs := make([]any, len(loopData)) itemErrors := make(map[int]string, len(loopData)) - r.logger.Debugf("Executing subworkflow...") + r.logger.Debugf("Executing subworkflow for step %s...", r.runID) wg := &sync.WaitGroup{} wg.Add(len(loopData)) errors := false @@ -471,7 +473,7 @@ func (r *runningStep) run() { }() } wg.Wait() - r.logger.Debugf("Subworkflow complete.") + r.logger.Debugf("Subworkflow %s complete.", r.runID) r.lock.Lock() previousStage := string(r.currentStage) r.currentState = step.RunningStepStateRunning diff --git a/internal/step/lifecycle.go b/internal/step/lifecycle.go index 83495250..e7ad97e3 100644 --- a/internal/step/lifecycle.go +++ b/internal/step/lifecycle.go @@ -73,6 +73,8 @@ type LifecycleStage struct { // It will automatically create a DAG node between the current and the described next stages to ensure // that it is running in order. NextStages []string + // RemovedStages describes stages that should be removed from the DAG if this stage is run. + RemovedStages []string // Fatal indicates that this stage should be treated as fatal unless handled by the workflow. Fatal bool } diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index a1708766..0a325eb0 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -3,8 +3,10 @@ package plugin import ( "context" "fmt" + "go.flow.arcalot.io/pluginsdk/plugin" "strings" "sync" + "time" log "go.arcalot.io/log/v2" "go.flow.arcalot.io/deployer" @@ -166,6 +168,7 @@ var crashedLifecycleStage = step.LifecycleStage{ FinishedName: "crashed", } +// Lifecycle returns a lifecycle that contains all plugin lifecycle stages func (p *pluginProvider) Lifecycle() step.Lifecycle[step.LifecycleStage] { return step.Lifecycle[step.LifecycleStage]{ InitialStage: string(StageIDDeploy), @@ -181,6 +184,8 @@ func (p *pluginProvider) Lifecycle() step.Lifecycle[step.LifecycleStage] { } } +// LoadSchema deploys the plugin, connects to the plugin's ATP server, loads its schema, then +// returns a runnableStep struct. Not to be confused with the runningStep struct. func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) (step.RunnableStep, error) { image := inputs["plugin"].(string) @@ -205,7 +210,7 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) } return &runnableStep{ - schemas: s, + schemas: *s, logger: p.logger, image: image, deployerRegistry: p.deployerRegistry, @@ -217,7 +222,7 @@ type runnableStep struct { image string deployerRegistry registry.Registry logger log.Logger - schemas schema.Schema[schema.Step] + schemas schema.SchemaSchema localDeployer deployer.Connector } @@ -261,6 +266,31 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st if !ok { return result, fmt.Errorf("the step '%s' does not exist in the '%s' plugin", stepID, r.image) } + + stopIfProperty := schema.NewPropertySchema( + schema.NewAnySchema(), + schema.NewDisplayValue( + schema.PointerTo("Stop condition"), + schema.PointerTo("If this field is filled with a non-false value, the step is cancelled (even if currently executing)."), + nil, + ), + false, + nil, + nil, + nil, + nil, + nil, + ) + // Now validate that the step's internal dependencies can be resolved (like stop_if's dependency on the cancel signal) + cancelSignal := stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] + if cancelSignal == nil { + // Not present + stopIfProperty.Disable(fmt.Sprintf("Cancel signal is not present in plugin image '%s', step '%s'", r.image, stepID)) + } else if err := plugin.CancellationSignalSchema.DataSchemaValue.ValidateCompatibility(cancelSignal.DataSchemaValue); err != nil { + // Present but incompatible + stopIfProperty.Disable(fmt.Sprintf("Cancel signal invalid schema in plugin image '%s', step '%s' (%s)", r.image, stepID, err)) + } + return step.Lifecycle[step.LifecycleStageWithSchema]{ InitialStage: "deploying", Stages: []step.LifecycleStageWithSchema{ @@ -350,20 +380,7 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st { LifecycleStage: cancelledLifecycleStage, InputSchema: map[string]*schema.PropertySchema{ - "stop_if": schema.NewPropertySchema( - schema.NewAnySchema(), - schema.NewDisplayValue( - schema.PointerTo("Stop condition"), - schema.PointerTo("If this field is filled with a non-false value, the step is cancelled (even if currently executing)."), - nil, - ), - false, - nil, - nil, - nil, - nil, - nil, - ), + "stop_if": stopIfProperty, }, Outputs: nil, }, @@ -403,7 +420,7 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st }, nil } -func (r *runnableStep) Start(input map[string]any, stageChangeHandler step.StageChangeHandler) (step.RunningStep, error) { +func (r *runnableStep) Start(input map[string]any, runID string, stageChangeHandler step.StageChangeHandler) (step.RunningStep, error) { rawStep, ok := input["step"] stepID := "" if ok && rawStep != nil { @@ -448,15 +465,17 @@ func (r *runnableStep) Start(input map[string]any, stageChangeHandler step.Stage lock: &sync.Mutex{}, ctx: ctx, cancel: cancel, - done: make(chan struct{}), deployInput: make(chan any, 1), runInput: make(chan any, 1), logger: r.logger, image: r.image, - step: stepID, + pluginStepID: stepID, state: step.RunningStepStateStarting, localDeployer: r.localDeployer, executionChannel: make(chan executionResult), + signalToStep: make(chan schema.Input), + signalFromStep: make(chan schema.Input), + runID: runID, } go s.run() @@ -471,20 +490,24 @@ type runningStep struct { lock *sync.Mutex ctx context.Context cancel context.CancelFunc - done chan struct{} deployInput chan any deployInputAvailable bool runInput chan any runInputAvailable bool logger log.Logger currentStage StageID + runID string // The ID associated with this execution (the workflow step ID) image string - step string + pluginStepID string // The ID of the step in the plugin state step.RunningStepState useLocalDeployer bool localDeployer deployer.Connector container deployer.Plugin executionChannel chan executionResult + signalToStep chan schema.Input // Communicates with the ATP client, not other steps. + signalFromStep chan schema.Input // Communicates with the ATP client, not other steps. + // Store channels for sending pre-calculated signal outputs to other steps? + // Store channels for receiving pre-calculated signal inputs from other steps? } func (r *runningStep) CurrentStage() string { @@ -540,10 +563,12 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro r.lock.Unlock() return fmt.Errorf("input provided more than once") } + // Ensure input is given if input["input"] == nil { r.lock.Unlock() return fmt.Errorf("bug: invalid input for 'running' stage, expected 'input' field") } + // Validate the input by unserializing it if _, err := r.stepSchema.Input().Unserialize(input["input"]); err != nil { r.lock.Unlock() return err @@ -564,8 +589,23 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro return nil case string(StageIDCancelled): if input["stop_if"] != false && input["stop_if"] != nil { - r.logger.Infof("Cancelling step %s", r.step) - r.cancel() // This should cancel the plugin deployment or execution. + r.logger.Infof("Cancelling step %s/%s", r.runID, r.pluginStepID) + + // Verify that the step has a cancel signal + cancelSignal := r.stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] + if cancelSignal == nil { + r.logger.Errorf("could not cancel step %s/%s. Does not contain cancel signal receiver.", r.runID, r.pluginStepID) + } else if err := plugin.CancellationSignalSchema.DataSchema().ValidateCompatibility(cancelSignal.DataSchema()); err != nil { + r.logger.Errorf("validation failed for cancel signal for step %s/%s: %s", r.runID, r.pluginStepID, err) + } else { + // Canceling the context should be enough when the stage isn't running. + if r.currentStage == StageIDRunning { + // Validated. Now call the signal. + r.signalToStep <- schema.Input{ID: cancelSignal.ID(), InputData: map[any]any{}} + } + } + // Now cancel the context to stop + r.cancel() } r.lock.Unlock() return nil @@ -594,14 +634,12 @@ func (r *runningStep) Close() error { } r.container = nil r.lock.Unlock() - <-r.done return nil } func (r *runningStep) run() { defer func() { r.cancel() - close(r.done) }() container, err := r.deployStage() if err != nil { @@ -612,7 +650,7 @@ func (r *runningStep) run() { select { case <-r.ctx.Done(): if err := container.Close(); err != nil { - r.logger.Warningf("failed to remove deployed container for step %s", r.step) + r.logger.Warningf("failed to remove deployed container for step %s/%s", r.runID, r.pluginStepID) } r.lock.Unlock() return @@ -620,6 +658,7 @@ func (r *runningStep) run() { r.container = container } r.lock.Unlock() + r.logger.Debugf("Successfully deployed container with ID '%s' for step %s/%s", container.ID(), r.runID, r.pluginStepID) if err := r.startStage(container); err != nil { r.startFailed(err) return @@ -630,6 +669,7 @@ func (r *runningStep) run() { } func (r *runningStep) deployStage() (deployer.Plugin, error) { + r.logger.Debugf("Deploying stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() if !r.deployInputAvailable { r.state = step.RunningStepStateWaitingForInput @@ -682,6 +722,7 @@ type executionResult struct { } func (r *runningStep) startStage(container deployer.Plugin) error { + r.logger.Debugf("Starting stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() previousStage := string(r.currentStage) r.currentStage = StageIDStarting @@ -724,25 +765,31 @@ func (r *runningStep) startStage(container deployer.Plugin) error { return err } steps := inputSchema.Steps() - stepSchema, ok := steps[r.step] + stepSchema, ok := steps[r.pluginStepID] if !ok { - return fmt.Errorf("schema mismatch between local and remote deployed plugin, no stepSchema named %s found in remote", r.step) + return fmt.Errorf("error in run step %s: schema mismatch between local and remote deployed plugin, no stepSchema named %s found in remote", r.runID, r.pluginStepID) } + // Re-verify input. This should have also been done earlier. if _, err := stepSchema.Input().Unserialize(runInput); err != nil { - return fmt.Errorf("schema mismatch between local and remote deployed plugin, unserializing input failed (%w)", err) + return fmt.Errorf("schema mismatch between local and remote deployed plugin in step %s/%s, unserializing input failed (%w)", r.runID, r.pluginStepID, err) } - // Runs the ATP client in a goroutine in order to wait for it or context done. - // On context done, the deployer tries to end execution. That will shut down - // (with sigterm) the container. Then wait for output, or error out. + // Runs the ATP client in a goroutine in order to wait for it. + // On context done, the deployer has 30 seconds before it will error out. go func() { - outputID, outputData, err := atpClient.Execute(r.step, runInput) + outputID, outputData, err := atpClient.Execute( + schema.Input{ID: r.pluginStepID, InputData: runInput}, + r.signalToStep, + r.signalFromStep, + ) r.executionChannel <- executionResult{outputID, outputData, err} + }() return nil } func (r *runningStep) runStage() error { + r.logger.Debugf("Running stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() previousStage := string(r.currentStage) r.currentStage = StageIDRunning @@ -765,12 +812,20 @@ func (r *runningStep) runStage() error { return result.err } case <-r.ctx.Done(): - // In this case, it is being instructed to stop. + // In this case, it is being instructed to stop. A signal should have been sent. // Shutdown (with sigterm) the container, then wait for the output (valid or error). - r.logger.Debugf("Running step context done before step run complete. Cancelling and waiting for result.") - r.cancel() - // If necessary, you can add a timeout here for shutdowns that take too long. - result = <-r.executionChannel + r.logger.Debugf("Got step context done before step run complete. Waiting up to 30 seconds for result.") + select { + case result = <-r.executionChannel: + // Successfully stopped before end of timeout. + case <-time.After(time.Duration(30) * time.Second): + r.logger.Warningf("Step %s/%s did not terminate within the 30 second time limit. Closing container.", + r.runID, r.pluginStepID) + if err := r.Close(); err != nil { + r.logger.Warningf("Error in step %s/%s while closing plugin container (%w)", r.runID, r.pluginStepID, err) + } + } + } // Execution complete, move to finished stage. @@ -806,6 +861,7 @@ func (r *runningStep) runStage() error { } func (r *runningStep) deployFailed(err error) { + r.logger.Debugf("Deploy failed stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() previousStage := string(r.currentStage) r.currentStage = StageIDDeployFailed @@ -822,7 +878,7 @@ func (r *runningStep) deployFailed(err error) { string(StageIDDeployFailed), false, ) - r.logger.Warningf("Plugin %s deploy failed. %v", r.step, err) + r.logger.Warningf("Plugin step %s/%s deploy failed. %v", r.runID, r.pluginStepID, err) // Now it's done. r.lock.Lock() @@ -843,6 +899,7 @@ func (r *runningStep) deployFailed(err error) { } func (r *runningStep) startFailed(err error) { + r.logger.Debugf("Start failed stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() previousStage := string(r.currentStage) r.currentStage = StageIDCrashed @@ -855,7 +912,7 @@ func (r *runningStep) startFailed(err error) { nil, string(r.currentStage), false) - r.logger.Warningf("Plugin step %s start failed. %v", r.step, err) + r.logger.Warningf("Plugin step %s/%s start failed. %v", r.runID, r.pluginStepID, err) // Now it's done. r.lock.Lock() @@ -876,6 +933,7 @@ func (r *runningStep) startFailed(err error) { } func (r *runningStep) runFailed(err error) { + r.logger.Debugf("Run failed stage for step %s/%s", r.runID, r.pluginStepID) // A current lack of observability into the atp client prevents // non-fragile testing of this function. @@ -894,7 +952,7 @@ func (r *runningStep) runFailed(err error) { string(r.currentStage), false) - r.logger.Warningf("Plugin step %s run failed. %v", r.step, err) + r.logger.Warningf("Plugin step %s/%s run failed. %v", r.runID, r.pluginStepID, err) // Now it's done. r.lock.Lock() diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index de3d797f..7e605672 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -182,11 +182,11 @@ func TestProvider_HappyError(t *testing.T) { } // start with a step id that is not in the schema - _, err = runnable.Start(map[string]any{"step": "wrong_stepid"}, handler) + _, err = runnable.Start(map[string]any{"step": "wrong_stepid"}, t.Name(), handler) assert.Error(t, err) // default step id - running, err := runnable.Start(map[string]any{"step": nil}, handler) + running, err := runnable.Start(map[string]any{"step": nil}, t.Name(), handler) assert.NoError(t, err) // non-existent stage @@ -290,7 +290,7 @@ func TestProvider_DeployFail(t *testing.T) { } // default step id - running, err := runnable.Start(map[string]any{"step": nil}, handler) + running, err := runnable.Start(map[string]any{"step": nil}, t.Name(), handler) assert.NoError(t, err) assert.NoError(t, running.ProvideStageInput( @@ -353,7 +353,7 @@ func TestProvider_StartFail(t *testing.T) { message: make(chan string), } - running, err := runnable.Start(map[string]any{"step": "wait"}, handler) + running, err := runnable.Start(map[string]any{"step": "wait"}, t.Name(), handler) assert.NoError(t, err) // tell deployer that this run should not succeed diff --git a/internal/step/provider.go b/internal/step/provider.go index 3e173675..9d77bae0 100644 --- a/internal/step/provider.go +++ b/internal/step/provider.go @@ -67,6 +67,7 @@ type RunnableStep interface { // match the RunSchema. Start( input map[string]any, + runID string, stageChangeHandler StageChangeHandler, ) (RunningStep, error) } diff --git a/workflow/any.go b/workflow/any.go index 14000d6e..0040fa3c 100644 --- a/workflow/any.go +++ b/workflow/any.go @@ -33,6 +33,15 @@ func (a *anySchemaWithExpressions) Validate(data any) error { return err } +func (a *anySchemaWithExpressions) ValidateCompatibility(dataOrType any) error { + // If expression, resolve it before calling ValidateCompatibility + if _, ok := dataOrType.(expressions.Expression); ok { + // Assume okay + return nil + } + return a.anySchema.ValidateCompatibility(dataOrType) +} + func (a *anySchemaWithExpressions) Serialize(data any) (any, error) { return a.checkAndConvert(data) } diff --git a/workflow/executor.go b/workflow/executor.go index 070afe51..6cba9c84 100644 --- a/workflow/executor.go +++ b/workflow/executor.go @@ -122,7 +122,15 @@ func (e *executor) Prepare(workflow *Workflow, workflowContext map[string][]byte return nil, err } - // Stage 5: The output data model + // Stage 5: Verify stage inputs + // Now that the output properties are here and the internal data model is here, it should be possible to loop + // through them again to verify that all inputs are valid. So verify that all required inputs are present, schemas + // are valid, etc. + // Do this by looping through the steps' inputs, then verifying that the dag can provide them. + if err := e.verifyStageInputs(workflow, workflowContext, stepLifecycles, dag, internalDataModel); err != nil { + return nil, err + } + // Stage 6: The output data model //goland:noinspection GoDeprecation if workflow.Output != nil { if len(workflow.Outputs) > 0 { @@ -277,6 +285,7 @@ func (e *executor) processSteps( nil, ) } + return runnableSteps, stepOutputProperties, stepLifecycles, stepRunData, nil } @@ -323,6 +332,135 @@ func (e *executor) connectStepDependencies( return nil } +// verifyStageInputs verifies the schemas of the step inputs +func (e *executor) verifyStageInputs( + workflow *Workflow, + workflowContext map[string][]byte, + stepLifecycles map[string]step.Lifecycle[step.LifecycleStageWithSchema], + dag dgraph.DirectedGraph[*DAGItem], + internalDataModel *schema.ScopeSchema, +) error { + for stepID, _ /*stepData*/ := range workflow.Steps { + lifecycle := stepLifecycles[stepID] + for _, stage := range lifecycle.Stages { + currentStageNode, err := dag.GetNodeByID(GetStageNodeID(stepID, stage.ID)) + if err != nil { + return fmt.Errorf("bug: node for current stage not found (%w)", err) + } + // stageData provides the info needed for this node, without the expressions resolved. + stageData := currentStageNode.Item().Data + + // Use reflection to convert the stage's input data to a readable map. + parsedInputs := make(map[string]any) + if stageData != nil { + v := reflect.ValueOf(stageData) + if v.Kind() != reflect.Map { + return fmt.Errorf("could not validate input. Stage data is not a map. It is %s", v.Kind()) + } + + for _, reflectedKey := range v.MapKeys() { + if reflectedKey.Kind() != reflect.Interface { + return fmt.Errorf("expected input key to be interface of a string. Got %s", reflectedKey.Kind()) + } + // Now convert interface to string + key, ok := reflectedKey.Interface().(string) + if !ok { + return fmt.Errorf("error converting input key to string") + } + value := v.MapIndex(reflectedKey).Interface() + parsedInputs[key] = value + } + } + // Next, loop through the input schema fields. + for name, stageInputSchema := range stage.InputSchema { + providedInputForField := parsedInputs[name] + // Check if the field is present in the stage data. + // If it is NOT present and is NOT required, continue to next field. + // If it is NOT present and IS required, fail + // If it IS present, verify whether schema is compatible with the schema of the provided data, + // then notify the provider that the data is present. + // This is running pre-workflow run, so you can check the schemas, but most fields won't be able to be + // resolved to an actual value. + if providedInputForField == nil { + // not present + if stageInputSchema.RequiredValue { + return fmt.Errorf("required input %s of type %s not found for step %s", + name, stageInputSchema.TypeID(), stepID) + } + } else { + // It is present, so make sure it is compatible. + err := e.preValidateCompatibility(internalDataModel, providedInputForField, stageInputSchema, workflowContext) + if err != nil { + return fmt.Errorf("input validation failed for workflow step %s stage %s (%w)", stepID, stage.ID, err) + } + } + } + } + } + return nil +} + +func (e *executor) preValidateCompatibility(rootSchema schema.Scope, inputField any, propertySchema *schema.PropertySchema, + workflowContext map[string][]byte) error { + // Get the type/value structure + inputTypeStructure, err := e.createTypeStructure(rootSchema, inputField, workflowContext) + if err != nil { + return err + } + // Now validate + return propertySchema.ValidateCompatibility(inputTypeStructure) +} + +// createTypeStructure generates a structure of all the type information of the input field. +// When the literal is known, it includes the original value. +// When the literal is not known, but the schema is, it includes the value. +// When it encounters a map or list, it preserves it and recursively continues. +func (e *executor) createTypeStructure(rootSchema schema.Scope, inputField any, workflowContext map[string][]byte) (any, error) { + + // Expression, so the exact value may not be known yet. So just get the type from it. + if expr, ok := inputField.(expressions.Expression); ok { + // Is expression, so evaluate it. + e.logger.Debugf("Evaluating expression %s...", expr.String()) + return expr.Type(rootSchema, workflowContext) + } + + v := reflect.ValueOf(inputField) + switch v.Kind() { + case reflect.Slice: + // Okay. Construct the list of schemas, and pass it into the + + result := make([]any, v.Len()) + for i := 0; i < v.Len(); i++ { + value := v.Index(i).Interface() + newValue, err := e.createTypeStructure(rootSchema, value, workflowContext) + if err != nil { + return nil, fmt.Errorf("failed to resolve expressions (%w)", err) + } + result[i] = newValue + } + return result, nil + case reflect.Map: + result := make(map[string]any, v.Len()) + for _, reflectedKey := range v.MapKeys() { + key := reflectedKey.Interface() + keyAsStr, ok := key.(string) + if !ok { + return nil, fmt.Errorf("failed to generate type structure. Key is not of type string") + } + value := v.MapIndex(reflectedKey).Interface() + newValue, err := e.createTypeStructure(rootSchema, value, workflowContext) + if err != nil { + return nil, fmt.Errorf("failed to resolve expressions (%w)", err) + } + result[keyAsStr] = newValue + } + return result, nil + default: + // Not an expression, so it's actually data. Just return the input + return inputField, nil + } +} + // buildInternalDataModel builds an internal data model that the expressions can query. func (e *executor) buildInternalDataModel(input schema.Scope, stepOutputProperties map[string]*schema.PropertySchema) *schema.ScopeSchema { internalDataModel := schema.NewScopeSchema( diff --git a/workflow/executor_test.go b/workflow/executor_test.go index 059f195c..fcb0eb81 100644 --- a/workflow/executor_test.go +++ b/workflow/executor_test.go @@ -3,16 +3,76 @@ package workflow_test import ( "context" "fmt" + "go.arcalot.io/assert" + "go.flow.arcalot.io/deployer" + deployerregistry "go.flow.arcalot.io/deployer/registry" "go.flow.arcalot.io/engine/config" + "go.flow.arcalot.io/engine/internal/step" + "go.flow.arcalot.io/engine/internal/step/plugin" + testimpl "go.flow.arcalot.io/testdeployer" "testing" "go.arcalot.io/lang" "go.arcalot.io/log/v2" "go.flow.arcalot.io/engine/internal/step/dummy" - "go.flow.arcalot.io/engine/internal/step/registry" + stepregistry "go.flow.arcalot.io/engine/internal/step/registry" "go.flow.arcalot.io/engine/workflow" ) +func getTestImplPreparedWorkflow(t *testing.T, workflowDefinition string) (workflow.ExecutableWorkflow, error) { + logger := log.NewLogger(log.LevelDebug, log.NewTestWriter(t)) + cfg := &config.Config{ + LoggedOutputConfigs: map[string]*config.StepOutputLogConfig{ + "terminated_early": { + LogLevel: log.LevelError, + }, + }, + } + stepRegistry := NewTestImplStepRegistry(logger, t) + + executor := lang.Must2(workflow.NewExecutor( + logger, + cfg, + stepRegistry, + )) + wf := assert.NoErrorR[*workflow.Workflow](t)(workflow.NewYAMLConverter(stepRegistry).FromYAML([]byte(workflowDefinition))) + return executor.Prepare(wf, map[string][]byte{}) +} + +func getDummyDeployerPreparedWorkflow(t *testing.T, workflowDefinition string) (workflow.ExecutableWorkflow, error) { + logger := log.NewLogger(log.LevelDebug, log.NewTestWriter(t)) + cfg := &config.Config{} + stepRegistry := assert.NoErrorR[step.Registry](t)(stepregistry.New( + dummy.New(), + )) + executor := assert.NoErrorR[workflow.Executor](t)(workflow.NewExecutor( + logger, + cfg, + stepRegistry, + )) + wf := assert.NoErrorR[*workflow.Workflow](t)(workflow.NewYAMLConverter(stepRegistry).FromYAML([]byte(workflowDefinition))) + return executor.Prepare(wf, map[string][]byte{}) +} + +func NewTestImplStepRegistry( + logger log.Logger, + t *testing.T, +) step.Registry { + deployerRegistry := deployerregistry.New( + deployer.Any(testimpl.NewFactory()), + ) + + pluginProvider := assert.NoErrorR[step.Provider](t)( + plugin.New(logger, deployerRegistry, map[string]interface{}{ + "type": "test-impl", + "deploy_time": "0", + }), + ) + return assert.NoErrorR[step.Registry](t)(stepregistry.New( + pluginProvider, + )) +} + var sharedInputWorkflowYAML = `--- input: root: RootObject @@ -26,6 +86,7 @@ input: steps: say_hi: kind: dummy + # Both name and nickname reference the same variable name: !expr $.input.name nickname: !expr $.input.name output: @@ -37,27 +98,9 @@ output: // and one step-output going into two step-outputs. // These cause duplicate connections to be made, which need to be handled properly. func TestSharedInput(t *testing.T) { - logger := log.NewLogger(log.LevelDebug, log.NewTestWriter(t)) - stepRegistry := lang.Must2(registry.New( - dummy.New(), - )) - - executor, err := workflow.NewExecutor(logger, &config.Config{}, stepRegistry) - if err != nil { - t.Fatalf("Failed to create Executor, %e", err) - } - - yamlConverter := workflow.NewYAMLConverter(stepRegistry) - decodedWorkflow, err := yamlConverter.FromYAML([]byte(sharedInputWorkflowYAML)) - if err != nil { - t.Fatalf("Failed to load workflow from YAML, %e", err) - } - - preparedWorkflow, err := executor.Prepare(decodedWorkflow, nil) - if err != nil { - t.Fatalf("Failed to prepare workflow, %e", err) - } - + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getDummyDeployerPreparedWorkflow(t, sharedInputWorkflowYAML), + ) ctx, cancel := context.WithCancel(context.Background()) defer cancel() outputID, outputData, err := preparedWorkflow.Execute(ctx, map[string]any{ @@ -69,3 +112,105 @@ func TestSharedInput(t *testing.T) { fmt.Printf("%s: %s\n", outputID, outputData.(map[any]any)["message"]) // Output: success: Hello Arca Lot! } + +var missingInputWorkflowDefinition1 = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + incomplete_wait: + plugin: "n/a" + step: wait + # Missing input +outputs: + a: + b: !expr $.steps.incomplete_wait.outputs +` + +var missingInputWorkflowDefinition2 = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + say_hi: + kind: dummy + # Missing name +outputs: + a: + b: !expr $.steps.say_hi.greet +` + +func TestMissingInput(t *testing.T) { + // For this test, a workflow's step will be missing its inputs. + _, err := getTestImplPreparedWorkflow(t, missingInputWorkflowDefinition1) + assert.Error(t, err) + + _, err = getDummyDeployerPreparedWorkflow(t, missingInputWorkflowDefinition2) + assert.Error(t, err) +} + +var mismatchedStepInputTypesWorkflowDefinition = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + wait_1: + plugin: "n/a" + step: wait + input: + wait_time_ms: 0 + wait_2: + plugin: "n/a" + step: wait + input: + # Should fail during preparation, due to message being a string, and wait_time_ms expecting an int + wait_time_ms: !expr $.steps.wait_1.outputs.success.message +outputs: + a: + b: !expr $.steps.wait_2.outputs +` + +func TestMismatchedStepInputTypes(t *testing.T) { + _, err := getTestImplPreparedWorkflow(t, mismatchedStepInputTypesWorkflowDefinition) + assert.Error(t, err) +} + +var mismatchedInputTypesWorkflowDefinition = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: + a: + display: + description: "Just for testing" + name: "a" + required: true + type: + type_id: string +steps: + wait_1: + plugin: "n/a" + step: wait + input: + # This is trying to put a string into an int field + wait_time_ms: !expr $.input.a +outputs: + a: + b: !expr $.steps.wait_1.outputs +` + +func TestMismatchedInputTypes(t *testing.T) { + _, err := getTestImplPreparedWorkflow(t, mismatchedInputTypesWorkflowDefinition) + assert.Error(t, err) +} diff --git a/workflow/workflow.go b/workflow/workflow.go index 044f9cb2..b77011c2 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -110,7 +110,7 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s }, } e.logger.Debugf("Launching step %s...", stepID) - runningStep, err := runnableStep.Start(e.stepRunData[stepID], stageHandler) + runningStep, err := runnableStep.Start(e.stepRunData[stepID], stepID, stageHandler) if err != nil { return "", nil, fmt.Errorf("failed to launch step %s (%w)", stepID, err) } diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index fb81dc37..64dfb0ed 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -3,21 +3,13 @@ package workflow_test import ( "context" "errors" + "go.arcalot.io/lang" + "go.arcalot.io/log/v2" + "go.flow.arcalot.io/engine/config" "testing" "time" - "go.flow.arcalot.io/deployer" - "go.flow.arcalot.io/engine/internal/step" - "go.flow.arcalot.io/engine/internal/step/plugin" - testimpl "go.flow.arcalot.io/testdeployer" - "go.arcalot.io/assert" - "go.arcalot.io/lang" - "go.arcalot.io/log/v2" - deployerregistry "go.flow.arcalot.io/deployer/registry" - "go.flow.arcalot.io/engine/config" - "go.flow.arcalot.io/engine/internal/step/dummy" - stepregistry "go.flow.arcalot.io/engine/internal/step/registry" "go.flow.arcalot.io/engine/workflow" ) @@ -40,26 +32,9 @@ output: ` func TestOutputFailed(t *testing.T) { - logConfig := log.Config{ - Level: log.LevelError, - Destination: log.DestinationStdout, - } - logger := log.New( - logConfig, + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getDummyDeployerPreparedWorkflow(t, badWorkflowDefinition), ) - cfg := &config.Config{ - Log: logConfig, - } - stepRegistry := lang.Must2(stepregistry.New( - dummy.New(), - )) - executor := lang.Must2(workflow.NewExecutor( - logger, - cfg, - stepRegistry, - )) - wf := lang.Must2(workflow.NewYAMLConverter(stepRegistry).FromYAML([]byte(badWorkflowDefinition))) - preparedWorkflow := lang.Must2(executor.Prepare(wf, map[string][]byte{})) _, outputData, err := preparedWorkflow.Execute(context.Background(), map[string]any{"name": "Arca Lot"}) assert.Nil(t, outputData) assert.Error(t, err) @@ -87,64 +62,183 @@ steps: plugin: "n/a" step: wait input: - wait_time_ms: 0 + # It needs to be long enough for it to ensure that long_wait is in a running state. + # The other case will be tested separately. + wait_time_ms: 20 outputs: a: cancelled_step_output: !expr $.steps.long_wait.outputs ` -func NewTestImplStepRegistry( - logger log.Logger, - t *testing.T, -) step.Registry { - deployerRegistry := deployerregistry.New( - deployer.Any(testimpl.NewFactory()), +func TestStepCancellation(t *testing.T) { + // For this test, a simple workflow will run wait steps, with one that's + // supposed to be stopped when the first stops. + // The long one will be long enough that there is no reasonable way + // for it to finish before the first step. + // The test double deployer will be used for this test, as we + // need a deployer to test the plugin step provider. + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getTestImplPreparedWorkflow(t, stepCancellationWorkflowDefinition), ) + outputID, outputData, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) + assert.NoError(t, err) + assert.Equals(t, outputID, "a") + stepResult := assert.MapContainsKeyAny(t, "cancelled_step_output", outputData.(map[any]any)) + assert.MapContainsKey(t, "cancelled_early", stepResult.(map[string]any)) +} - pluginProvider := assert.NoErrorR[step.Provider](t)( - plugin.New(logger, deployerRegistry, map[string]interface{}{ - "type": "test-impl", - "deploy_time": "0", - }), +var earlyStepCancellationWorkflowDefinition = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + # This one needs to run longer than the total time expected of all the other steps, with + # a large enough difference to prevent timing errors breaking the test. + end_wait: + plugin: "n/a" + step: wait + input: + wait_time_ms: 80 + # Delay needs to be delayed long enough to ensure that last_step isn't running when it's cancelled by short_wait + delay: + plugin: "n/a" + step: wait + input: + wait_time_ms: 50 + last_step: + plugin: "n/a" + step: wait + input: + wait_time_ms: 0 + # Delay it so it doesn't run, and gets cancelled before deployment. + wait_for: !expr $.steps.delay.outputs + # You can verify that this test works by commenting out this line. It should fail. + stop_if: !expr $.steps.short_wait.outputs + short_wait: + plugin: "n/a" + step: wait + input: + # End the test quickly. + wait_time_ms: 0 +outputs: + # If not properly cancelled, fail_case will have output. + fail_case: + unattainable: !expr $.steps.last_step.outputs + correct_case: + a: !expr $.steps.end_wait.outputs +` + +func TestEarlyStepCancellation(t *testing.T) { + // For this test, a simple workflow will run wait steps, with the workflow + // The long one will be long enough that there is no reasonable way + // for it to finish before the first step. + // The test double deployer will be used for this test, as we + // need a deployer to test the plugin step provider. + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getTestImplPreparedWorkflow(t, earlyStepCancellationWorkflowDefinition), ) - return assert.NoErrorR[step.Registry](t)(stepregistry.New( - pluginProvider, - )) + startTime := time.Now() // Right before execute to not include pre-processing time. + outputID, _, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) + duration := time.Since(startTime) + t.Logf("Test execution time: %s", duration) + // A nil value means the output could not be constructed, which is intended due to us cancelling the step it depends on. + // If it's not nil, that means the step didn't get cancelled. + assert.NoError(t, err) + assert.Equals(t, outputID, "correct_case") + // All steps that can result in output are 0 ms, so just leave some time for processing. + assert.LessThan(t, duration.Milliseconds(), 200) } -func TestStepCancellation(t *testing.T) { - // For this test, a simple workflow will run wait steps, with one that's - // supposed to be stopped when the first stops. +var deploymentStepCancellationWorkflowDefinition = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + # This one needs to run longer than the total time expected of all the other steps, with + # a large enough difference to prevent timing errors breaking the test. + end_wait: + plugin: "n/a" + step: wait + input: + wait_time_ms: 100 + step_to_cancel: + plugin: "n/a" + step: wait + input: + wait_time_ms: 0 + # You can verify that this test works by commenting out this line. It should fail. + stop_if: !expr $.steps.short_wait.outputs + # Delay needs to be delayed long enough to ensure that it's in a deploy state when it's cancelled by short_wait + deploy: + type: "test-impl" + deploy_time: 50 # 5 ms + short_wait: + plugin: "n/a" + step: wait + input: + # End the test quickly. + wait_time_ms: 0 +outputs: + # If not properly cancelled, fail_case will have output. + fail_case: + unattainable: !expr $.steps.step_to_cancel.outputs + correct_case: + a: !expr $.steps.end_wait.outputs +` + +func TestDeploymentStepCancellation(t *testing.T) { + // For this test, a simple workflow will run wait steps, with the workflow // The long one will be long enough that there is no reasonable way // for it to finish before the first step. // The test double deployer will be used for this test, as we // need a deployer to test the plugin step provider. - logConfig := log.Config{ - Level: log.LevelInfo, - Destination: log.DestinationStdout, - } - logger := log.New( - logConfig, + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getTestImplPreparedWorkflow(t, deploymentStepCancellationWorkflowDefinition), ) - cfg := &config.Config{ - Log: logConfig, - } - stepRegistry := NewTestImplStepRegistry(logger, t) + startTime := time.Now() // Right before execute to not include pre-processing time. + outputID, _, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) + duration := time.Since(startTime) + t.Logf("Test execution time: %s", duration) + // A nil value means the output could not be constructed, which is intended due to us cancelling the step it depends on. + // If it's not nil, that means the step didn't get cancelled. + assert.NoError(t, err) + assert.Equals(t, outputID, "correct_case") + // All steps that can result in output are 0 ms, so just leave some time for processing. + assert.LessThan(t, duration.Milliseconds(), 200) +} - executor := lang.Must2(workflow.NewExecutor( - logger, - cfg, - stepRegistry, - )) - wf := lang.Must2(workflow.NewYAMLConverter(stepRegistry).FromYAML([]byte(stepCancellationWorkflowDefinition))) - preparedWorkflow := lang.Must2(executor.Prepare(wf, map[string][]byte{})) - outputID, outputData, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) +var simpleValidLiteralInputWaitWorkflowDefinition = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + wait_1: + plugin: "n/a" + step: wait + input: + wait_time_ms: 0 +outputs: + a: + b: !expr $.steps.wait_1.outputs +` + +func TestSimpleValidWaitWorkflow(t *testing.T) { + // Just a single wait + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getTestImplPreparedWorkflow(t, simpleValidLiteralInputWaitWorkflowDefinition), + ) + outputID, _, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) assert.NoError(t, err) assert.Equals(t, outputID, "a") - stepResult := outputData.(map[interface{}]interface{})["cancelled_step_output"] - assert.NotNil(t, stepResult) - stepResultCancelledEarly := stepResult.(map[string]interface{})["cancelled_early"] - assert.NotNil(t, stepResultCancelledEarly) } var waitForSerialWorkflowDefinition = ` @@ -180,7 +274,6 @@ func TestWaitForSerial(t *testing.T) { // as each step runs for 5s and are run serially // The test double deployer will be used for this test, as we // need a deployer to test the plugin step provider. - startTime := time.Now() logConfig := log.Config{ Level: log.LevelInfo, Destination: log.DestinationStdout, @@ -200,6 +293,7 @@ func TestWaitForSerial(t *testing.T) { )) wf := lang.Must2(workflow.NewYAMLConverter(stepRegistry).FromYAML([]byte(waitForSerialWorkflowDefinition))) preparedWorkflow := lang.Must2(executor.Prepare(wf, map[string][]byte{})) + startTime := time.Now() // Right before execute to not include pre-processing time. outputID, outputData, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) assert.NoError(t, err) assert.Equals(t, outputID, "success") @@ -239,18 +333,18 @@ steps: plugin: "n/a" step: wait input: - wait_time_ms: 5000 + wait_time_ms: 500 second_wait: plugin: "n/a" step: wait input: - wait_time_ms: 5000 + wait_time_ms: 500 wait_for: !expr $.steps.first_wait.outputs.success third_wait: plugin: "n/a" step: wait input: - wait_time_ms: 5000 + wait_time_ms: 500 wait_for: !expr $.steps.first_wait.outputs.success outputs: success: @@ -265,7 +359,6 @@ func TestWaitForParallel(t *testing.T) { // as the first step runs for 5s and other two steps run in parallel after the first succeeds // The test double deployer will be used for this test, as we // need a deployer to test the plugin step provider. - startTime := time.Now() logConfig := log.Config{ Level: log.LevelInfo, Destination: log.DestinationStdout, @@ -285,6 +378,7 @@ func TestWaitForParallel(t *testing.T) { )) wf := lang.Must2(workflow.NewYAMLConverter(stepRegistry).FromYAML([]byte(waitForParallelWorkflowDefinition))) preparedWorkflow := lang.Must2(executor.Prepare(wf, map[string][]byte{})) + startTime := time.Now() // Right before execute to not include pre-processing time. outputID, outputData, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) assert.NoError(t, err) assert.Equals(t, outputID, "success") @@ -297,7 +391,7 @@ func TestWaitForParallel(t *testing.T) { duration := time.Since(startTime) t.Logf("Test execution time: %s", duration) var waitSuccess bool - if duration > 10*time.Second && duration < 20*time.Second { + if duration > 1*time.Second && duration < 2*time.Second { waitSuccess = true t.Logf("Steps second_wait and third_wait are running in parallel after waiting for the first_wait step.") } else { From 5e5659ae050d28930f19799d0bf951322f28bc0e Mon Sep 17 00:00:00 2001 From: jaredoconnell Date: Fri, 4 Aug 2023 10:18:27 -0400 Subject: [PATCH 02/54] Updated go mod/sum --- go.mod | 4 ++-- go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 1e58398b..74522c44 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module go.flow.arcalot.io/engine go 1.18 require ( - go.arcalot.io/assert v1.3.0 + go.arcalot.io/assert v1.4.0 go.arcalot.io/dgraph v1.1.0 go.arcalot.io/lang v1.0.0 go.arcalot.io/log/v2 v2.0.0 @@ -11,7 +11,7 @@ require ( go.flow.arcalot.io/dockerdeployer v0.3.0 go.flow.arcalot.io/expressions v0.2.0 go.flow.arcalot.io/kubernetesdeployer v0.5.1 - go.flow.arcalot.io/pluginsdk v0.3.0-beta.1 + go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7 go.flow.arcalot.io/podmandeployer v0.3.1 go.flow.arcalot.io/pythondeployer v0.1.2 go.flow.arcalot.io/testdeployer v0.2.0 diff --git a/go.sum b/go.sum index bb6b1d7a..e2d3aa22 100644 --- a/go.sum +++ b/go.sum @@ -143,8 +143,8 @@ github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1: github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -go.arcalot.io/assert v1.3.0 h1:+uQex4s9gezATpTyFxUY5dlAcrwI1Me5fSmdcydGHho= -go.arcalot.io/assert v1.3.0/go.mod h1:Xy3ScX0p9IMY89gdsgexOKxnmDr0nGHG9dV7p8Uxg7w= +go.arcalot.io/assert v1.4.0 h1:X7ptZSl4WrB+jUn53Kzvp3aPWCSsCgUp4iW2wdLEOv4= +go.arcalot.io/assert v1.4.0/go.mod h1:Xy3ScX0p9IMY89gdsgexOKxnmDr0nGHG9dV7p8Uxg7w= go.arcalot.io/dgraph v1.1.0 h1:c0LR7+xdUy7Ki6e4nR9rBvK0Upr4Nu49fu+poP/9WMg= go.arcalot.io/dgraph v1.1.0/go.mod h1:FuNv92OgHsJYepD6Unwn+S/4DioBnv06JxQ2BtQct7E= go.arcalot.io/lang v1.0.0 h1:mgDaieT4wWdZTnR4V7+/pgYRmzfU7VZZgIzHccuxAbY= @@ -159,8 +159,8 @@ go.flow.arcalot.io/expressions v0.2.0 h1:2k8InnpLqVmv5SDvJ1xRz1ubqF+zKN44U08D50T go.flow.arcalot.io/expressions v0.2.0/go.mod h1:m4p6oCwjjxRjPCAYGeZDNP9DGfoDODHo1z4pBoO+Lpc= go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEXBAnyoWJnHls= go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= -go.flow.arcalot.io/pluginsdk v0.3.0-beta.1 h1:RrC5SKDkhwG/enE/FajAxRF1izET61/LO4lhaI9q094= -go.flow.arcalot.io/pluginsdk v0.3.0-beta.1/go.mod h1:7cEk8LSxpZakyfrmKTPbiMhlrZvWtCPYcaI7qfSu8MM= +go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7 h1:tCib+S6+Shof0GdMMBwFco5EoDDlqMRc4qe9Ydbvy+s= +go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7/go.mod h1:sk4Vj6E25YhfrG7+hIg0T9AZ/zygv0vPqiFF5OCEJAk= go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= go.flow.arcalot.io/podmandeployer v0.3.1/go.mod h1:SmROc9nHG+KfKasyTeKtGmI9EBlXCupXjBIgX5glGn8= go.flow.arcalot.io/pythondeployer v0.1.2 h1:IUIoYTuw+2N1zrpSv8yaO3wsS6aWAFwbLlYNhCEM+zI= From 25fa40e3fe980e7e8d5044c03de90f5961b89a53 Mon Sep 17 00:00:00 2001 From: jaredoconnell Date: Wed, 6 Sep 2023 14:00:43 -0400 Subject: [PATCH 03/54] Progress towards debugging deadlock detection Mostly this is test cases, and some mild refactoring that should have a similar result --- cmd/run-plugin/run.go | 18 ++--- go.mod | 2 +- go.sum | 2 + internal/step/plugin/provider.go | 111 ++++++++++++++++++++++++------- internal/step/provider.go | 4 +- workflow/model.go | 23 ++++++- workflow/workflow.go | 88 +++++++++++++++--------- workflow/workflow_test.go | 75 ++++++++++++++++++++- 8 files changed, 251 insertions(+), 72 deletions(-) diff --git a/cmd/run-plugin/run.go b/cmd/run-plugin/run.go index 32d230ca..f80e72df 100644 --- a/cmd/run-plugin/run.go +++ b/cmd/run-plugin/run.go @@ -65,11 +65,11 @@ func main() { default: panic("No deployer or invalid deployer selected. Options: docker, podman, testimpl. Select with -deployer") } - - connector, err := d.Create(defaultConfig, log.New(log.Config{ + logger := log.New(log.Config{ Level: log.LevelDebug, Destination: log.DestinationStdout, - })) + }) + connector, err := d.Create(defaultConfig, logger) if err != nil { panic(err) } @@ -83,19 +83,19 @@ func main() { go func() { select { case <-ctrlC: - fmt.Println("Received CTRL-C. Sending cancel signal...") + logger.Infof("Received CTRL-C. Sending cancel signal...") toStepSignals <- schema.Input{ ID: plugin.CancellationSignalSchema.ID(), InputData: make(map[string]any), } - fmt.Println("Signal sent.") + logger.Infof("Signal request sent to ATP client.") cancel() case <-ctx.Done(): // Done here. } }() - fmt.Println("Deploying") + logger.Infof("Deploying") plugin, err := connector.Deploy(ctx, image) if err != nil { panic(err) @@ -106,8 +106,8 @@ func main() { } }() - atpClient := atp.NewClient(plugin) - fmt.Println("Getting schema") + atpClient := atp.NewClientWithLogger(plugin, logger) + logger.Infof("Getting schema") pluginSchema, err := atpClient.ReadSchema() if err != nil { panic(err) @@ -131,7 +131,7 @@ func main() { fmt.Printf("Running step %s\n", stepID) outputID, outputData, err := atpClient.Execute( schema.Input{ID: stepID, InputData: input}, - nil, + toStepSignals, nil, ) output := map[string]any{ diff --git a/go.mod b/go.mod index 74522c44..118d8491 100644 --- a/go.mod +++ b/go.mod @@ -11,7 +11,7 @@ require ( go.flow.arcalot.io/dockerdeployer v0.3.0 go.flow.arcalot.io/expressions v0.2.0 go.flow.arcalot.io/kubernetesdeployer v0.5.1 - go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7 + go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1 go.flow.arcalot.io/podmandeployer v0.3.1 go.flow.arcalot.io/pythondeployer v0.1.2 go.flow.arcalot.io/testdeployer v0.2.0 diff --git a/go.sum b/go.sum index e2d3aa22..dd7f5499 100644 --- a/go.sum +++ b/go.sum @@ -161,6 +161,8 @@ go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEX go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7 h1:tCib+S6+Shof0GdMMBwFco5EoDDlqMRc4qe9Ydbvy+s= go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7/go.mod h1:sk4Vj6E25YhfrG7+hIg0T9AZ/zygv0vPqiFF5OCEJAk= +go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1 h1:l6kMeJPGacZp9e1rJKi5CARdCGV4NGN6LlXvn+cItxM= +go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1/go.mod h1:sk4Vj6E25YhfrG7+hIg0T9AZ/zygv0vPqiFF5OCEJAk= go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= go.flow.arcalot.io/podmandeployer v0.3.1/go.mod h1:SmROc9nHG+KfKasyTeKtGmI9EBlXCupXjBIgX5glGn8= go.flow.arcalot.io/pythondeployer v0.1.2 h1:IUIoYTuw+2N1zrpSv8yaO3wsS6aWAFwbLlYNhCEM+zI= diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 0a325eb0..eb517f4f 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "go.flow.arcalot.io/pluginsdk/plugin" + "reflect" "strings" "sync" "time" @@ -285,7 +286,8 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st cancelSignal := stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] if cancelSignal == nil { // Not present - stopIfProperty.Disable(fmt.Sprintf("Cancel signal is not present in plugin image '%s', step '%s'", r.image, stepID)) + stopIfProperty.Disable(fmt.Sprintf("Cancel signal with ID '%s' is not present in plugin image '%s', step '%s'. Signal handler IDs present: %v", + plugin.CancellationSignalSchema.ID(), r.image, stepID, reflect.ValueOf(stepSchema.SignalHandlers()).MapKeys())) } else if err := plugin.CancellationSignalSchema.DataSchemaValue.ValidateCompatibility(cancelSignal.DataSchemaValue); err != nil { // Present but incompatible stopIfProperty.Disable(fmt.Sprintf("Cancel signal invalid schema in plugin image '%s', step '%s' (%s)", r.image, stepID, err)) @@ -513,13 +515,15 @@ type runningStep struct { func (r *runningStep) CurrentStage() string { r.lock.Lock() defer r.lock.Unlock() - return string(r.currentStage) + tempStage := string(r.currentStage) + return tempStage } func (r *runningStep) State() step.RunningStepState { r.lock.Lock() defer r.lock.Unlock() - return r.state + tempState := r.state + return tempState } func (r *runningStep) ProvideStageInput(stage string, input map[string]any) error { @@ -527,6 +531,8 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro // affect the counting of step states in the workflow's Execute function // and notifySteps function. r.lock.Lock() + r.logger.Debugf("ProvideStageInput START") + defer r.logger.Debugf("ProvideStageInput END") // Checks which stage it is getting input for switch stage { @@ -554,9 +560,16 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro if r.state == step.RunningStepStateWaitingForInput && r.currentStage == StageIDDeploy { r.state = step.RunningStepStateRunning } - r.lock.Unlock() + + // TODO: CHECK IF THIS IS OKAY // Feed the deploy step its input. - r.deployInput <- unserializedDeployerConfig + select { + case r.deployInput <- unserializedDeployerConfig: + default: + r.lock.Unlock() + return fmt.Errorf("unable to provide input to deploy stage for step %s/%s", r.runID, r.pluginStepID) + } + r.lock.Unlock() return nil case string(StageIDStarting): if r.runInputAvailable { @@ -575,14 +588,22 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro } // Make sure we transition the state before unlocking so there are no race conditions. r.runInputAvailable = true - if r.state == step.RunningStepStateWaitingForInput && r.currentStage == StageIDStarting { - r.state = step.RunningStepStateRunning - } + //if r.state == step.RunningStepStateWaitingForInput && r.currentStage == StageIDDeploy { + // r.logger.Debugf("Input available. State set to running.") + // r.state = step.RunningStepStateRunning + //} // Unlock before passing the data over the channel to prevent a deadlock. // The other end of the channel needs to be unlocked to read the data. - r.lock.Unlock() + + // TODO: CHECK IF THIS WORKS // Feed the run step its input over the channel. - r.runInput <- input["input"] + select { + case r.runInput <- input["input"]: + default: + r.lock.Unlock() + return fmt.Errorf("unable to provide input to run stage for step %s/%s", r.runID, r.pluginStepID) + } + r.lock.Unlock() return nil case string(StageIDRunning): r.lock.Unlock() @@ -672,7 +693,9 @@ func (r *runningStep) deployStage() (deployer.Plugin, error) { r.logger.Debugf("Deploying stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() if !r.deployInputAvailable { - r.state = step.RunningStepStateWaitingForInput + r.logger.Debugf("PROCESSING inputs state while deploying.") + //r.state = step.RunningStepStateWaitingForInput + r.state = step.RunningStepStateRunning } else { r.state = step.RunningStepStateRunning } @@ -690,15 +713,29 @@ func (r *runningStep) deployStage() (deployer.Plugin, error) { var deployerConfig any var useLocalDeployer bool + // First, non-blocking retrieval select { case deployerConfig = <-r.deployInput: r.lock.Lock() r.state = step.RunningStepStateRunning - useLocalDeployer = r.useLocalDeployer r.lock.Unlock() - case <-r.ctx.Done(): - return nil, fmt.Errorf("step closed before deployment config could be obtained") + default: // Default, so it doesn't block on this receive + // It's waiting now. + r.lock.Lock() + r.state = step.RunningStepStateWaitingForInput + r.lock.Unlock() + select { + case deployerConfig = <-r.deployInput: + r.lock.Lock() + r.state = step.RunningStepStateRunning + r.lock.Unlock() + case <-r.ctx.Done(): + return nil, fmt.Errorf("step closed before deployment config could be obtained") + } } + r.lock.Lock() + useLocalDeployer = r.useLocalDeployer + r.lock.Unlock() var stepDeployer = r.localDeployer if !useLocalDeployer { @@ -726,12 +763,26 @@ func (r *runningStep) startStage(container deployer.Plugin) error { r.lock.Lock() previousStage := string(r.currentStage) r.currentStage = StageIDStarting + inputRecievedEarly := false - if !r.runInputAvailable { - r.state = step.RunningStepStateWaitingForInput - } else { + var runInput any + select { + case runInput = <-r.runInput: + // Good. It received it immediately. r.state = step.RunningStepStateRunning + inputRecievedEarly = true + default: // The default makes it not wait. + r.state = step.RunningStepStateWaitingForInput } + + //if !r.runInputAvailable { + // r.logger.Debugf("Waiting for input state while starting 1.") + // //r.state = step.RunningStepStateWaitingForInput + // // TEMP: Assume running until stage change + // r.state = step.RunningStepStateRunning + //} else { + // r.state = step.RunningStepStateRunning + //} runInputAvailable := r.runInputAvailable r.lock.Unlock() @@ -746,17 +797,29 @@ func (r *runningStep) startStage(container deployer.Plugin) error { r.lock.Lock() r.currentStage = StageIDStarting - r.state = step.RunningStepStateWaitingForInput + r.logger.Debugf("Waiting for input state while starting 2.") r.lock.Unlock() - var runInput any - select { - case runInput = <-r.runInput: + // First, try to non-blocking retrieve the runInput. + // If not yet available, set to state waiting for input and do a blocking receive. + // If it is available, continue. + if !inputRecievedEarly { + // Input is not yet available. Now waiting. r.lock.Lock() - r.state = step.RunningStepStateRunning + if r.state != step.RunningStepStateWaitingForInput { + r.logger.Warningf("State not waiting for input when receiving from channel.") + } r.lock.Unlock() - case <-r.ctx.Done(): - return fmt.Errorf("step closed while waiting for run configuration") + + // Do a blocking wait for input now. + select { + case runInput = <-r.runInput: + r.lock.Lock() + r.state = step.RunningStepStateRunning + r.lock.Unlock() + case <-r.ctx.Done(): + return fmt.Errorf("step closed while waiting for run configuration") + } } atpClient := atp.NewClientWithLogger(container, r.logger) diff --git a/internal/step/provider.go b/internal/step/provider.go index 9d77bae0..aa252116 100644 --- a/internal/step/provider.go +++ b/internal/step/provider.go @@ -42,7 +42,7 @@ type StageChangeHandler interface { previousStageOutputID *string, previousStageOutput *any, newStage string, - waitingForInput bool, + inputAvailable bool, ) // OnStepComplete is called when the step has completed a final stage in its lifecycle and communicates the output. @@ -82,7 +82,7 @@ const ( RunningStepStateWaitingForInput RunningStepState = "waiting_for_input" // RunningStepStateRunning indicates that the step is working. RunningStepStateRunning RunningStepState = "running" - // RunningStepStateFinished indicates that the step has finished. + // RunningStepStateFinished indicates that the step has finished, including failure cases. RunningStepStateFinished RunningStepState = "finished" ) diff --git a/workflow/model.go b/workflow/model.go index a3534e93..e1398605 100644 --- a/workflow/model.go +++ b/workflow/model.go @@ -210,7 +210,7 @@ type ErrNoMorePossibleSteps struct { // Error returns an explanation on why the error happened. func (e ErrNoMorePossibleSteps) Error() string { - var outputs []string //nolint:prealloc + var outputsUnmetDependencies []string //nolint:prealloc for _, node := range e.dag.ListNodes() { if node.Item().Kind != DAGItemKindOutput { continue @@ -223,7 +223,24 @@ func (e ErrNoMorePossibleSteps) Error() string { for i := range inbound { unmetDependencies = append(unmetDependencies, i) } - outputs = append(outputs, fmt.Sprintf("%s: %s", node.Item().OutputID, strings.Join(unmetDependencies, ", "))) + outputsUnmetDependencies = append( + outputsUnmetDependencies, + fmt.Sprintf("%s: %s", node.Item().OutputID, strings.Join(unmetDependencies, ", ")), + ) } - return fmt.Sprintf("no steps running, no more executable steps, cannot construct any output (outputs have the following dependencies: %s)", strings.Join(outputs, "; ")) + return fmt.Sprintf( + "no steps running, no more executable steps, cannot construct any output (outputs have the following dependencies: %s)", + strings.Join(outputsUnmetDependencies, "; "), + ) +} + +// ErrInvalidState indicates that the workflow failed due to an invalid state. +type ErrInvalidState struct { + processingSteps int + msg string +} + +// Error returns an explanation on why the error happened. +func (e ErrInvalidState) Error() string { + return fmt.Sprintf("Workflow failed due to invalid state (%s). Processing steps: %d", e.msg, e.processingSteps) } diff --git a/workflow/workflow.go b/workflow/workflow.go index b77011c2..1ded9aeb 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -5,6 +5,7 @@ import ( "context" "fmt" "reflect" + "runtime" "sync" "go.flow.arcalot.io/engine/config" @@ -75,6 +76,8 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s workflowContext: e.workflowContext, } + l.lock.Lock() + // Iterate over all steps to set them up with proper handlers, then launch them. // Even though they're launched, the workflow won't execute until the input is provided. for stepID, runnableStep := range e.runnableSteps { @@ -92,13 +95,14 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s l.data["steps"].(map[string]any)[stepID] = stepDataModel var stageHandler step.StageChangeHandler = &stageChangeHandler{ - onStageChange: func(step step.RunningStep, previousStage *string, previousStageOutputID *string, previousStageOutput *any, stage string, waitingForInput bool) { + onStageChange: func(step step.RunningStep, previousStage *string, previousStageOutputID *string, previousStageOutput *any, stage string, inputAvailable bool) { waitingForInputText := "" - if waitingForInput { + if !inputAvailable { waitingForInputText = " and is waiting for input" } - e.logger.Debugf("Stage change for step %s to %s%s...", stepID, stage, waitingForInputText) + e.logger.Debugf("START Stage change for step %s to %s%s...", stepID, stage, waitingForInputText) l.onStageComplete(stepID, previousStage, previousStageOutputID, previousStageOutput) + e.logger.Debugf("DONE Stage change for step %s to %s%s...", stepID, stage, waitingForInputText) }, onStepComplete: func(step step.RunningStep, previousStage string, previousStageOutputID *string, previousStageOutput *any) { if previousStageOutputID != nil { @@ -116,6 +120,7 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s } l.runningSteps[stepID] = runningStep } + l.lock.Unlock() // Let's make sure we are closing all steps once this function terminates so we don't leave stuff running. defer func() { e.logger.Debugf("Terminating all steps...") @@ -204,7 +209,12 @@ type loopState struct { func (l *loopState) onStageComplete(stepID string, previousStage *string, previousStageOutputID *string, previousStageOutput *any) { l.lock.Lock() - defer l.lock.Unlock() + defer func() { + if previousStage != nil { + l.checkForDeadlocks() + } + l.lock.Unlock() + }() if previousStage == nil { return @@ -216,6 +226,7 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo l.cancel() return } + l.logger.Debugf("Removed node '%s' from the DAG", stageNode.ID()) if err := stageNode.Remove(); err != nil { l.logger.Errorf("Failed to remove stage node ID %s (%w)", stageNode.ID(), err) l.lastError = fmt.Errorf("failed to remove stage node ID %s (%w)", stageNode.ID(), err) @@ -230,6 +241,8 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo l.cancel() return } + // Removes the node from the DAG. This results in the nodes not having inbound connections, allowing them to be processed. + l.logger.Debugf("Removed node '%s' from the DAG", outputNode.ID()) if err := outputNode.Remove(); err != nil { l.logger.Errorf("Failed to remove output node ID %s (%w)", outputNode.ID(), err) l.lastError = fmt.Errorf("failed to remove output node ID %s (%w)", outputNode.ID(), err) @@ -257,7 +270,7 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo // notifySteps is a function we can call to go through all DAG nodes that have no inbound connections and // provide step inputs based on expressions. -// The lock should be acquired when this is called. +// The lock should be acquired by the caller before this is called. func (l *loopState) notifySteps() { //nolint:gocognit // This function goes through the DAG and feeds the input to all steps that have no further inbound // dependencies. @@ -308,27 +321,27 @@ func (l *loopState) notifySteps() { //nolint:gocognit } // This check is here just to make sure it has the required fields set - if node.Item().StepID != "" && node.Item().StageID != "" { - stageInputData := untypedInputData.(map[any]any) - typedInputData := make(map[string]any, len(stageInputData)) - for k, v := range stageInputData { - typedInputData[k.(string)] = v - } - // Sends it to the plugin - l.logger.Debugf("Providing stage input for %s...", nodeID) - if err := l.runningSteps[node.Item().StepID].ProvideStageInput( - node.Item().StageID, - typedInputData, - ); err != nil { - l.logger.Errorf("Bug: failed to provide input to step %s (%w)", node.Item().StepID, err) - l.lastError = fmt.Errorf("bug: failed to provide input to step %s (%w)", node.Item().StepID, err) - l.cancel() - return - } - } else { + if node.Item().StepID == "" || node.Item().StageID == "" { // This shouldn't happen panic("Step or stage ID missing") } + + stageInputData := untypedInputData.(map[any]any) + typedInputData := make(map[string]any, len(stageInputData)) + for k, v := range stageInputData { + typedInputData[k.(string)] = v + } + // Sends it to the plugin + l.logger.Debugf("Providing stage input for %s...", nodeID) + if err := l.runningSteps[node.Item().StepID].ProvideStageInput( + node.Item().StageID, + typedInputData, + ); err != nil { + l.logger.Errorf("Bug: failed to provide input to step %s (%w)", node.Item().StepID, err) + l.lastError = fmt.Errorf("bug: failed to provide input to step %s (%w)", node.Item().StepID, err) + l.cancel() + return + } case DAGItemKindOutput: // We have received enough data to construct the workflow output. l.logger.Debugf("Constructing workflow output.") @@ -354,17 +367,22 @@ func (l *loopState) notifySteps() { //nolint:gocognit } } } +} + +func (l *loopState) checkForDeadlocks() { // Here we make sure we don't have a deadlock. counters := struct { - starting int - waiting int - running int - finished int + starting int + waitingWithInbound int + waitingWithoutInbound int + running int + finished int }{ 0, 0, 0, 0, + 0, } for stepID, runningStep := range l.runningSteps { switch runningStep.State() { @@ -372,20 +390,25 @@ func (l *loopState) notifySteps() { //nolint:gocognit counters.starting++ l.logger.Debugf("Step %s is currently starting.", stepID) case step.RunningStepStateWaitingForInput: - counters.waiting++ - connectionsMsg := "" dagNode, err := l.dag.GetNodeByID(GetStageNodeID(stepID, runningStep.CurrentStage())) switch { case err != nil: l.logger.Warningf("Failed to get DAG node for the debug message (%w)", err) + counters.waitingWithInbound++ case dagNode == nil: l.logger.Warningf("Failed to get DAG node for the debug message. Returned nil", err) + counters.waitingWithInbound++ default: inboundConnections, err := dagNode.ListInboundConnections() if err != nil { l.logger.Warningf("Error while listing inbound connections. (%w)", err) } + if len(inboundConnections) > 0 { + counters.waitingWithInbound++ + } else { + counters.waitingWithoutInbound++ + } i := 0 for k := range inboundConnections { @@ -406,13 +429,14 @@ func (l *loopState) notifySteps() { //nolint:gocognit } } l.logger.Infof( - "There are currently %d steps starting, %d waiting for input, %d running, %d finished", + "There are currently %d steps starting, %d waiting for input, %d ready for input, %d running, %d finished", counters.starting, - counters.waiting, + counters.waitingWithInbound, + counters.waitingWithoutInbound, counters.running, counters.finished, ) - if counters.starting == 0 && counters.running == 0 && !l.outputDone { + if counters.starting == 0 && counters.running == 0 && counters.waitingWithoutInbound == 0 && !l.outputDone { l.lastError = &ErrNoMorePossibleSteps{ l.dag, } diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index 64dfb0ed..26283675 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -177,7 +177,7 @@ steps: # Delay needs to be delayed long enough to ensure that it's in a deploy state when it's cancelled by short_wait deploy: type: "test-impl" - deploy_time: 50 # 5 ms + deploy_time: 50 # 50 ms short_wait: plugin: "n/a" step: wait @@ -400,3 +400,76 @@ func TestWaitForParallel(t *testing.T) { } assert.Equals(t, waitSuccess, true) } + +var missingInputsFailedDeploymentWorkflowDefinition = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + wait_1: + plugin: "n/a" + step: wait + input: + wait_time_ms: 0 + deploy: + type: "test-impl" + #deploy_time: 20000 # 10 ms + deploy_succeed: false + wait_2: + plugin: "n/a" + step: wait + wait_for: !expr $.steps.wait_1.outputs.success + input: + wait_time_ms: 0 +outputs: + a: + b: !expr $.steps.wait_2.outputs +` + +func TestMissingInputsFailedDeployment(t *testing.T) { + // For this test, the workflow should fail, not deadlock, due to no inputs possible. + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getTestImplPreparedWorkflow(t, missingInputsFailedDeploymentWorkflowDefinition), + ) + outputID, _, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) + assert.Error(t, err) + assert.Equals(t, outputID, "") +} + +var missingInputsWrongOutputWorkflowDefinition = ` +input: + root: RootObject + objects: + RootObject: + id: RootObject + properties: {} +steps: + wait_1: + plugin: "n/a" + step: wait + input: + wait_time_ms: 0 + wait_2: + plugin: "n/a" + step: wait + # No stop_if, so this shouldn't happen. + wait_for: !expr $.steps.wait_1.outputs.cancelled_early + input: + wait_time_ms: 0 +outputs: + a: + b: !expr $.steps.wait_2.outputs +` + +func TestMissingInputsWrongOutput(t *testing.T) { + // For this test, the workflow should fail, not deadlock, due to no inputs possible. + preparedWorkflow := assert.NoErrorR[workflow.ExecutableWorkflow](t)( + getTestImplPreparedWorkflow(t, missingInputsWrongOutputWorkflowDefinition), + ) + outputID, _, err := preparedWorkflow.Execute(context.Background(), map[string]any{}) + assert.Error(t, err) + assert.Equals(t, outputID, "") +} From 8188317931c7a602543a15e91c1c67d43c7d4b9d Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Thu, 14 Sep 2023 15:18:26 -0400 Subject: [PATCH 04/54] Use image that supports ARM64 --- engine_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine_test.go b/engine_test.go index cfe4906d..2d7bbbdd 100644 --- a/engine_test.go +++ b/engine_test.go @@ -133,7 +133,7 @@ func TestE2E(t *testing.T) { type_id: string steps: example: - plugin: quay.io/arcalot/arcaflow-plugin-template-python:0.1.0 + plugin: quay.io/arcalot/arcaflow-plugin-template-python:0.2.1 input: name: !expr $.input.name output: @@ -163,7 +163,7 @@ func TestE2EMultipleOutputs(t *testing.T) { type_id: string steps: example: - plugin: quay.io/arcalot/arcaflow-plugin-template-python:0.1.0 + plugin: quay.io/arcalot/arcaflow-plugin-template-python:0.2.1 input: name: !expr $.input.name outputs: From e715adda5da68082708a3b677b07ac3c829304f2 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Thu, 14 Sep 2023 15:39:22 -0400 Subject: [PATCH 05/54] Ensure that run finishes before closing the step This ensures that the goroutines are done when the engine exits. This fixes a test case, and will be important when retrying is added to the deadlock check --- internal/step/plugin/provider.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index eb517f4f..e77446bb 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -490,6 +490,7 @@ type runningStep struct { stepSchema schema.Step stageChangeHandler step.StageChangeHandler lock *sync.Mutex + wg sync.WaitGroup ctx context.Context cancel context.CancelFunc deployInput chan any @@ -655,12 +656,16 @@ func (r *runningStep) Close() error { } r.container = nil r.lock.Unlock() + // Wait for the run to finish to ensure that it's not running after closing. + r.wg.Wait() return nil } func (r *runningStep) run() { + r.wg.Add(1) // Wait for the run to finish before closing. defer func() { - r.cancel() + r.cancel() // Close before WaitGroup done + r.wg.Done() // Done. Close may now exit. }() container, err := r.deployStage() if err != nil { From ac0ef7e81088e531a6371d4bdfb65a0e8dc26439 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Thu, 14 Sep 2023 16:39:41 -0400 Subject: [PATCH 06/54] Added deadlock check retries This is a workaround for the situation where all of the steps are processing at the same time, all waiting for input --- workflow/workflow.go | 26 ++++++++++++++++++-------- 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/workflow/workflow.go b/workflow/workflow.go index 1ded9aeb..65eaa29b 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -5,8 +5,8 @@ import ( "context" "fmt" "reflect" - "runtime" "sync" + "time" "go.flow.arcalot.io/engine/config" @@ -211,7 +211,7 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo l.lock.Lock() defer func() { if previousStage != nil { - l.checkForDeadlocks() + l.checkForDeadlocks(3) } l.lock.Unlock() }() @@ -369,7 +369,7 @@ func (l *loopState) notifySteps() { //nolint:gocognit } } -func (l *loopState) checkForDeadlocks() { +func (l *loopState) checkForDeadlocks(retries int) { // Here we make sure we don't have a deadlock. counters := struct { starting int @@ -437,12 +437,22 @@ func (l *loopState) checkForDeadlocks() { counters.finished, ) if counters.starting == 0 && counters.running == 0 && counters.waitingWithoutInbound == 0 && !l.outputDone { - l.lastError = &ErrNoMorePossibleSteps{ - l.dag, + if retries <= 0 { + l.lastError = &ErrNoMorePossibleSteps{ + l.dag, + } + l.logger.Errorf("%v", l.lastError) + l.logger.Debugf("DAG:\n%s", l.dag.Mermaid()) + l.cancel() + } else { + // Retry. There are times when all the steps are in a transition state. + // Retrying will delay the check until after they are done with the transition. + l.logger.Warningf("No running steps. Rechecking...") + go func() { + time.Sleep(5 * time.Millisecond) + l.checkForDeadlocks(retries - 1) + }() } - l.logger.Errorf("%v", l.lastError) - l.logger.Debugf("DAG:\n%s", l.dag.Mermaid()) - l.cancel() } } From c5696ec28d46f3e6b20d1948cd05166065781199 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Thu, 14 Sep 2023 16:40:38 -0400 Subject: [PATCH 07/54] Update test cases to take less time Also improved error logic --- workflow/workflow_test.go | 33 +++++++++++++-------------------- 1 file changed, 13 insertions(+), 20 deletions(-) diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index 26283675..64fd1603 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -3,13 +3,13 @@ package workflow_test import ( "context" "errors" + "go.arcalot.io/assert" "go.arcalot.io/lang" "go.arcalot.io/log/v2" "go.flow.arcalot.io/engine/config" "testing" "time" - "go.arcalot.io/assert" "go.flow.arcalot.io/engine/workflow" ) @@ -253,12 +253,13 @@ steps: plugin: "n/a" step: wait input: - wait_time_ms: 500 + # Note: 5ms left only a 2.5ms margin for error. 10ms left almost 6ms. So 10ms min is recommended. + wait_time_ms: 10 second_wait: plugin: "n/a" step: wait input: - wait_time_ms: 500 + wait_time_ms: 10 wait_for: !expr $.steps.first_wait.outputs.success outputs: success: @@ -308,15 +309,11 @@ func TestWaitForSerial(t *testing.T) { duration := time.Since(startTime) t.Logf("Test execution time: %s", duration) - var waitSuccess bool - if duration >= 1*time.Second { - waitSuccess = true - t.Logf("Test execution time is greater than 1 second, steps are running serially due to the wait_for condition.") + if duration >= 20*time.Millisecond { + t.Logf("Test execution time is greater than 20 milliseconds; steps are correctly running serially due to the wait_for condition.") } else { - waitSuccess = false - t.Logf("Test execution time is lesser than 1 seconds, steps are not running serially.") + t.Fatalf("Test execution time is less than 20 milliseconds; steps are not running serially.") } - assert.Equals(t, waitSuccess, true) } // Running parallel steps which wait on the same previous step sometimes causes a race condition. This needs to be investigated. @@ -333,18 +330,18 @@ steps: plugin: "n/a" step: wait input: - wait_time_ms: 500 + wait_time_ms: 10 second_wait: plugin: "n/a" step: wait input: - wait_time_ms: 500 + wait_time_ms: 10 wait_for: !expr $.steps.first_wait.outputs.success third_wait: plugin: "n/a" step: wait input: - wait_time_ms: 500 + wait_time_ms: 10 wait_for: !expr $.steps.first_wait.outputs.success outputs: success: @@ -390,15 +387,11 @@ func TestWaitForParallel(t *testing.T) { duration := time.Since(startTime) t.Logf("Test execution time: %s", duration) - var waitSuccess bool - if duration > 1*time.Second && duration < 2*time.Second { - waitSuccess = true - t.Logf("Steps second_wait and third_wait are running in parallel after waiting for the first_wait step.") + if duration > 20*time.Millisecond && duration < 40*time.Millisecond { + t.Logf("Steps second_wait and third_wait are correctly running in parallel after waiting for the first_wait step.") } else { - waitSuccess = false - t.Logf("Steps second_wait and third_wait are not running in parallel.") + t.Fatalf("Steps second_wait and third_wait are not running in parallel.") } - assert.Equals(t, waitSuccess, true) } var missingInputsFailedDeploymentWorkflowDefinition = ` From 4c72b39a31c364f14e6f99b4c135b27a553c9481 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Thu, 14 Sep 2023 16:41:03 -0400 Subject: [PATCH 08/54] Update dependencies Required for prior commits to work --- go.mod | 4 +++- go.sum | 12 ++++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 118d8491..9ace3f4e 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,8 @@ module go.flow.arcalot.io/engine go 1.18 require ( - go.arcalot.io/assert v1.4.0 + github.com/stretchr/testify v1.8.1 + go.arcalot.io/assert v1.5.0 go.arcalot.io/dgraph v1.1.0 go.arcalot.io/lang v1.0.0 go.arcalot.io/log/v2 v2.0.0 @@ -47,6 +48,7 @@ require ( github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.0.2 // indirect github.com/pkg/errors v0.9.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect github.com/x448/float16 v0.8.4 // indirect go.flow.arcalot.io/testplugin v0.1.0 // indirect golang.org/x/mod v0.11.0 // indirect diff --git a/go.sum b/go.sum index dd7f5499..a176a195 100644 --- a/go.sum +++ b/go.sum @@ -133,9 +133,14 @@ github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasO github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= @@ -143,8 +148,8 @@ github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1: github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -go.arcalot.io/assert v1.4.0 h1:X7ptZSl4WrB+jUn53Kzvp3aPWCSsCgUp4iW2wdLEOv4= -go.arcalot.io/assert v1.4.0/go.mod h1:Xy3ScX0p9IMY89gdsgexOKxnmDr0nGHG9dV7p8Uxg7w= +go.arcalot.io/assert v1.5.0 h1:6Ifu7jBpHA6s1aEzf8nlv51MXyBoqDkLzmbvjoZpcXM= +go.arcalot.io/assert v1.5.0/go.mod h1:Xy3ScX0p9IMY89gdsgexOKxnmDr0nGHG9dV7p8Uxg7w= go.arcalot.io/dgraph v1.1.0 h1:c0LR7+xdUy7Ki6e4nR9rBvK0Upr4Nu49fu+poP/9WMg= go.arcalot.io/dgraph v1.1.0/go.mod h1:FuNv92OgHsJYepD6Unwn+S/4DioBnv06JxQ2BtQct7E= go.arcalot.io/lang v1.0.0 h1:mgDaieT4wWdZTnR4V7+/pgYRmzfU7VZZgIzHccuxAbY= @@ -159,8 +164,6 @@ go.flow.arcalot.io/expressions v0.2.0 h1:2k8InnpLqVmv5SDvJ1xRz1ubqF+zKN44U08D50T go.flow.arcalot.io/expressions v0.2.0/go.mod h1:m4p6oCwjjxRjPCAYGeZDNP9DGfoDODHo1z4pBoO+Lpc= go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEXBAnyoWJnHls= go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= -go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7 h1:tCib+S6+Shof0GdMMBwFco5EoDDlqMRc4qe9Ydbvy+s= -go.flow.arcalot.io/pluginsdk v0.3.0-beta.1.0.20230803174348-19bdaeed3fa7/go.mod h1:sk4Vj6E25YhfrG7+hIg0T9AZ/zygv0vPqiFF5OCEJAk= go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1 h1:l6kMeJPGacZp9e1rJKi5CARdCGV4NGN6LlXvn+cItxM= go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1/go.mod h1:sk4Vj6E25YhfrG7+hIg0T9AZ/zygv0vPqiFF5OCEJAk= go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= @@ -288,6 +291,7 @@ gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= From 54503aee9b1e90a2ae3887dc7846a9891f6f67ca Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Thu, 14 Sep 2023 18:25:21 -0400 Subject: [PATCH 09/54] Update go SDK to dev branch This adds the done message to the SDK --- go.mod | 4 ++-- go.sum | 3 +++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 9ace3f4e..63cb8302 100644 --- a/go.mod +++ b/go.mod @@ -12,7 +12,7 @@ require ( go.flow.arcalot.io/dockerdeployer v0.3.0 go.flow.arcalot.io/expressions v0.2.0 go.flow.arcalot.io/kubernetesdeployer v0.5.1 - go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1 + go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e go.flow.arcalot.io/podmandeployer v0.3.1 go.flow.arcalot.io/pythondeployer v0.1.2 go.flow.arcalot.io/testdeployer v0.2.0 @@ -28,7 +28,7 @@ require ( github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/emicklei/go-restful/v3 v3.10.0 // indirect - github.com/fxamacker/cbor/v2 v2.4.0 // indirect + github.com/fxamacker/cbor/v2 v2.5.0 // indirect github.com/go-logr/logr v1.2.3 // indirect github.com/go-openapi/jsonpointer v0.19.5 // indirect github.com/go-openapi/jsonreference v0.20.0 // indirect diff --git a/go.sum b/go.sum index a176a195..fce81044 100644 --- a/go.sum +++ b/go.sum @@ -40,6 +40,7 @@ github.com/evanphx/json-patch v4.12.0+incompatible h1:4onqiflcdA9EOZ4RxV643DvftH github.com/flowstack/go-jsonschema v0.1.1/go.mod h1:yL7fNggx1o8rm9RlgXv7hTBWxdBM0rVwpMwimd3F3N0= github.com/fxamacker/cbor/v2 v2.4.0 h1:ri0ArlOR+5XunOP8CRUowT0pSJOwhW098ZCUyskZD88= github.com/fxamacker/cbor/v2 v2.4.0/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= +github.com/fxamacker/cbor/v2 v2.5.0/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= @@ -166,6 +167,8 @@ go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEX go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1 h1:l6kMeJPGacZp9e1rJKi5CARdCGV4NGN6LlXvn+cItxM= go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1/go.mod h1:sk4Vj6E25YhfrG7+hIg0T9AZ/zygv0vPqiFF5OCEJAk= +go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e h1:Wh4RQdU9x1GNL8e0gppx9kKmjKC05QHt3eA8rssp2S0= +go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e/go.mod h1:8Gp8Q0Eo0s+2I4nNl3uQ95jPi2zFKEX7JqyjhmK7BiI= go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= go.flow.arcalot.io/podmandeployer v0.3.1/go.mod h1:SmROc9nHG+KfKasyTeKtGmI9EBlXCupXjBIgX5glGn8= go.flow.arcalot.io/pythondeployer v0.1.2 h1:IUIoYTuw+2N1zrpSv8yaO3wsS6aWAFwbLlYNhCEM+zI= From 9ad0570fcc8d2a7f5991f9f5bbc72a3bd49c2e6a Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Thu, 14 Sep 2023 18:38:35 -0400 Subject: [PATCH 10/54] Updated test deployer to dev branch --- go.mod | 2 +- go.sum | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 63cb8302..30e6c700 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e go.flow.arcalot.io/podmandeployer v0.3.1 go.flow.arcalot.io/pythondeployer v0.1.2 - go.flow.arcalot.io/testdeployer v0.2.0 + go.flow.arcalot.io/testdeployer v0.2.1-0.20230914223701-544954f47aee gopkg.in/yaml.v3 v3.0.1 ) diff --git a/go.sum b/go.sum index fce81044..71b061fc 100644 --- a/go.sum +++ b/go.sum @@ -40,6 +40,7 @@ github.com/evanphx/json-patch v4.12.0+incompatible h1:4onqiflcdA9EOZ4RxV643DvftH github.com/flowstack/go-jsonschema v0.1.1/go.mod h1:yL7fNggx1o8rm9RlgXv7hTBWxdBM0rVwpMwimd3F3N0= github.com/fxamacker/cbor/v2 v2.4.0 h1:ri0ArlOR+5XunOP8CRUowT0pSJOwhW098ZCUyskZD88= github.com/fxamacker/cbor/v2 v2.4.0/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= +github.com/fxamacker/cbor/v2 v2.5.0 h1:oHsG0V/Q6E/wqTS2O1Cozzsy69nqCiguo5Q1a1ADivE= github.com/fxamacker/cbor/v2 v2.5.0/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -175,6 +176,8 @@ go.flow.arcalot.io/pythondeployer v0.1.2 h1:IUIoYTuw+2N1zrpSv8yaO3wsS6aWAFwbLlYN go.flow.arcalot.io/pythondeployer v0.1.2/go.mod h1:vCkwB72TinFVb367/o0djptTvR+V004i1I5OQUeCcPU= go.flow.arcalot.io/testdeployer v0.2.0 h1:4/cLr58/e6o5ouVRuJ5hM28nhciwJrL9AOE5Sdb7rN0= go.flow.arcalot.io/testdeployer v0.2.0/go.mod h1:vy3Iu+9SHmugvOJRtMWAj8R+SE9BYi7k9Xi7DM5n7eQ= +go.flow.arcalot.io/testdeployer v0.2.1-0.20230914223701-544954f47aee h1:ZtvcstJb4Lh+1EWJvwuBATV1jWivzFIl+TzJRoDXAbA= +go.flow.arcalot.io/testdeployer v0.2.1-0.20230914223701-544954f47aee/go.mod h1:nvz66pthT7pCYN0NMOfIErNlbNz/FcSfmIQ4kSCXBS0= go.flow.arcalot.io/testplugin v0.1.0 h1:I2BT978XISjaSnQbpaJfmjo2cTmTeBV7q+1IwTGbrig= go.flow.arcalot.io/testplugin v0.1.0/go.mod h1:RsEWotEbX4irH+OM/d3dUOZMhHeBDDhqUymIKqZlESU= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= From 1770f1929a930da688c34a0b35e29312937e9745 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Mon, 18 Sep 2023 16:21:58 -0400 Subject: [PATCH 11/54] Updated deployers, and added python deployer to run-plugin --- cmd/run-plugin/run.go | 20 +++++++++++++++++++- go.mod | 8 +++----- go.sum | 24 ++++++------------------ 3 files changed, 28 insertions(+), 24 deletions(-) diff --git a/cmd/run-plugin/run.go b/cmd/run-plugin/run.go index f80e72df..b75537a9 100644 --- a/cmd/run-plugin/run.go +++ b/cmd/run-plugin/run.go @@ -9,6 +9,7 @@ import ( "go.flow.arcalot.io/pluginsdk/plugin" "go.flow.arcalot.io/pluginsdk/schema" podman "go.flow.arcalot.io/podmandeployer" + pythondeployer "go.flow.arcalot.io/pythondeployer" "os" "os/signal" @@ -23,14 +24,18 @@ func main() { var image string var file string var stepID string + var pythonPath string var d deployer.AnyConnectorFactory var defaultConfig any var deployerID = "docker" + var workingDir string flag.StringVar(&image, "image", image, "Docker image to run") flag.StringVar(&file, "file", file, "Input file") flag.StringVar(&stepID, "step", stepID, "Step name") flag.StringVar(&deployerID, "deployer", stepID, "The name of the deployer") + flag.StringVar(&pythonPath, "pythonpath", "", "Path to the Python environment") + flag.StringVar(&workingDir, "workingdir", "~/", "Path to store cloned repositories") flag.Parse() switch deployerID { @@ -62,8 +67,20 @@ func main() { if err != nil { panic(err) } + case "python": + pythonFactory := pythondeployer.NewFactory() + d = deployer.Any(pythonFactory) + configSchema := pythonFactory.ConfigurationSchema() + var err error + configInput := map[string]any{} + configInput["pythonPath"] = pythonPath + configInput["workdir"] = workingDir + defaultConfig, err = configSchema.UnserializeType(configInput) + if err != nil { + panic(err) + } default: - panic("No deployer or invalid deployer selected. Options: docker, podman, testimpl. Select with -deployer") + panic("No deployer or invalid deployer selected. Options: docker, podman, testimpl, python. Select with -deployer") } logger := log.New(log.Config{ Level: log.LevelDebug, @@ -98,6 +115,7 @@ func main() { logger.Infof("Deploying") plugin, err := connector.Deploy(ctx, image) if err != nil { + logger.Errorf("Error while deploying: %s", err) panic(err) } defer func() { diff --git a/go.mod b/go.mod index 30e6c700..1773a7a3 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,6 @@ module go.flow.arcalot.io/engine go 1.18 require ( - github.com/stretchr/testify v1.8.1 go.arcalot.io/assert v1.5.0 go.arcalot.io/dgraph v1.1.0 go.arcalot.io/lang v1.0.0 @@ -14,8 +13,8 @@ require ( go.flow.arcalot.io/kubernetesdeployer v0.5.1 go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e go.flow.arcalot.io/podmandeployer v0.3.1 - go.flow.arcalot.io/pythondeployer v0.1.2 - go.flow.arcalot.io/testdeployer v0.2.1-0.20230914223701-544954f47aee + go.flow.arcalot.io/pythondeployer v0.1.4 + go.flow.arcalot.io/testdeployer v0.2.1-0.20230914224521-f83e94942739 gopkg.in/yaml.v3 v3.0.1 ) @@ -48,9 +47,8 @@ require ( github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.0.2 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/pmezard/go-difflib v1.0.0 // indirect github.com/x448/float16 v0.8.4 // indirect - go.flow.arcalot.io/testplugin v0.1.0 // indirect + go.flow.arcalot.io/testplugin v0.1.1-0.20230914224236-656dd83abb0e // indirect golang.org/x/mod v0.11.0 // indirect golang.org/x/net v0.11.0 // indirect golang.org/x/oauth2 v0.2.0 // indirect diff --git a/go.sum b/go.sum index 71b061fc..4afc3999 100644 --- a/go.sum +++ b/go.sum @@ -38,8 +38,6 @@ github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/evanphx/json-patch v4.12.0+incompatible h1:4onqiflcdA9EOZ4RxV643DvftH5pOlLGNtQ5lPWQu84= github.com/flowstack/go-jsonschema v0.1.1/go.mod h1:yL7fNggx1o8rm9RlgXv7hTBWxdBM0rVwpMwimd3F3N0= -github.com/fxamacker/cbor/v2 v2.4.0 h1:ri0ArlOR+5XunOP8CRUowT0pSJOwhW098ZCUyskZD88= -github.com/fxamacker/cbor/v2 v2.4.0/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/fxamacker/cbor/v2 v2.5.0 h1:oHsG0V/Q6E/wqTS2O1Cozzsy69nqCiguo5Q1a1ADivE= github.com/fxamacker/cbor/v2 v2.5.0/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= @@ -135,14 +133,9 @@ github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasO github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= -github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= @@ -166,20 +159,16 @@ go.flow.arcalot.io/expressions v0.2.0 h1:2k8InnpLqVmv5SDvJ1xRz1ubqF+zKN44U08D50T go.flow.arcalot.io/expressions v0.2.0/go.mod h1:m4p6oCwjjxRjPCAYGeZDNP9DGfoDODHo1z4pBoO+Lpc= go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEXBAnyoWJnHls= go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= -go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1 h1:l6kMeJPGacZp9e1rJKi5CARdCGV4NGN6LlXvn+cItxM= -go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230830170647-4f3b981c11a1/go.mod h1:sk4Vj6E25YhfrG7+hIg0T9AZ/zygv0vPqiFF5OCEJAk= go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e h1:Wh4RQdU9x1GNL8e0gppx9kKmjKC05QHt3eA8rssp2S0= go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e/go.mod h1:8Gp8Q0Eo0s+2I4nNl3uQ95jPi2zFKEX7JqyjhmK7BiI= go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= go.flow.arcalot.io/podmandeployer v0.3.1/go.mod h1:SmROc9nHG+KfKasyTeKtGmI9EBlXCupXjBIgX5glGn8= -go.flow.arcalot.io/pythondeployer v0.1.2 h1:IUIoYTuw+2N1zrpSv8yaO3wsS6aWAFwbLlYNhCEM+zI= -go.flow.arcalot.io/pythondeployer v0.1.2/go.mod h1:vCkwB72TinFVb367/o0djptTvR+V004i1I5OQUeCcPU= -go.flow.arcalot.io/testdeployer v0.2.0 h1:4/cLr58/e6o5ouVRuJ5hM28nhciwJrL9AOE5Sdb7rN0= -go.flow.arcalot.io/testdeployer v0.2.0/go.mod h1:vy3Iu+9SHmugvOJRtMWAj8R+SE9BYi7k9Xi7DM5n7eQ= -go.flow.arcalot.io/testdeployer v0.2.1-0.20230914223701-544954f47aee h1:ZtvcstJb4Lh+1EWJvwuBATV1jWivzFIl+TzJRoDXAbA= -go.flow.arcalot.io/testdeployer v0.2.1-0.20230914223701-544954f47aee/go.mod h1:nvz66pthT7pCYN0NMOfIErNlbNz/FcSfmIQ4kSCXBS0= -go.flow.arcalot.io/testplugin v0.1.0 h1:I2BT978XISjaSnQbpaJfmjo2cTmTeBV7q+1IwTGbrig= -go.flow.arcalot.io/testplugin v0.1.0/go.mod h1:RsEWotEbX4irH+OM/d3dUOZMhHeBDDhqUymIKqZlESU= +go.flow.arcalot.io/pythondeployer v0.1.4 h1:A4YXgfzrzJZLp19kK8PV5B+ZG6MieiYSPILZSqDMUqE= +go.flow.arcalot.io/pythondeployer v0.1.4/go.mod h1:vCkwB72TinFVb367/o0djptTvR+V004i1I5OQUeCcPU= +go.flow.arcalot.io/testdeployer v0.2.1-0.20230914224521-f83e94942739 h1:uGz0wU4FXB3Llfk8mWsVP+BQGMTE2LJdfFOaRDxFfuo= +go.flow.arcalot.io/testdeployer v0.2.1-0.20230914224521-f83e94942739/go.mod h1:6oX0Kx7zwMxEdB/6h7LoIPiTTYgVYS3FJ0MdqlH6i6k= +go.flow.arcalot.io/testplugin v0.1.1-0.20230914224236-656dd83abb0e h1:xb0swZIsOerWcN2/76HnzQhJvJkL+1NUgnKw7EqhArc= +go.flow.arcalot.io/testplugin v0.1.1-0.20230914224236-656dd83abb0e/go.mod h1:HBjElWQmsQeBWPTBNFc1EHAatWFMG13CuanhCmJ/0Jw= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -297,7 +286,6 @@ gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= From 2fe30bee5a010fecad90309b2cf17899bf9157d8 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Mon, 18 Sep 2023 17:15:11 -0400 Subject: [PATCH 12/54] Label loggers --- cmd/arcaflow/main.go | 2 +- internal/step/foreach/provider.go | 2 +- internal/step/plugin/provider.go | 6 +++--- workflow/executor.go | 2 +- workflow/workflow.go | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cmd/arcaflow/main.go b/cmd/arcaflow/main.go index 83cea9af..967ebf90 100644 --- a/cmd/arcaflow/main.go +++ b/cmd/arcaflow/main.go @@ -133,7 +133,7 @@ Options: } cfg.Log.Stdout = os.Stderr - logger := log.New(cfg.Log) + logger := log.New(cfg.Log).WithLabel("source", "main") dirContext, err := loadContext(dir) if err != nil { diff --git a/internal/step/foreach/provider.go b/internal/step/foreach/provider.go index a8defac3..f8aaa71f 100644 --- a/internal/step/foreach/provider.go +++ b/internal/step/foreach/provider.go @@ -19,7 +19,7 @@ func New( executorFactory func(logger log.Logger) (workflow.Executor, error), ) (step.Provider, error) { return &forEachProvider{ - logger: logger, + logger: logger.WithLabel("source", "foreach-provider"), yamlParserFactory: yamlParserFactory, executorFactory: executorFactory, }, nil diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index e77446bb..9ec7c93e 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -31,12 +31,12 @@ func New(logger log.Logger, deployerRegistry registry.Registry, localDeployerCon if err != nil { return nil, fmt.Errorf("failed to load local deployer configuration, please check your Arcaflow configuration file (%w)", err) } - localDeployer, err := deployerRegistry.Create(unserializedLocalDeployerConfig, logger) + localDeployer, err := deployerRegistry.Create(unserializedLocalDeployerConfig, logger.WithLabel("source", "deployer")) if err != nil { return nil, fmt.Errorf("invalid local deployer configuration, please check your Arcaflow configuration file (%w)", err) } return &pluginProvider{ - logger: logger, + logger: logger.WithLabel("source", "plugin-provider"), deployerRegistry: deployerRegistry, localDeployer: localDeployer, }, nil @@ -745,7 +745,7 @@ func (r *runningStep) deployStage() (deployer.Plugin, error) { var stepDeployer = r.localDeployer if !useLocalDeployer { var err error - stepDeployer, err = r.deployerRegistry.Create(deployerConfig, r.logger) + stepDeployer, err = r.deployerRegistry.Create(deployerConfig, r.logger.WithLabel("source", "deployer")) if err != nil { return nil, err } diff --git a/workflow/executor.go b/workflow/executor.go index 6cba9c84..af97f7bd 100644 --- a/workflow/executor.go +++ b/workflow/executor.go @@ -30,7 +30,7 @@ func NewExecutor( return nil, fmt.Errorf("bug: no step registry passed to NewExecutor") } return &executor{ - logger: logger, + logger: logger.WithLabel("source", "executor"), stepRegistry: stepRegistry, config: config, }, nil diff --git a/workflow/workflow.go b/workflow/workflow.go index 65eaa29b..0d107926 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -60,7 +60,7 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s defer cancel() l := &loopState{ - logger: e.logger, + logger: e.logger.WithLabel("source", "workflow"), config: e.config, lock: &sync.Mutex{}, data: map[string]any{ From cb57e6c70fb22e563947ee43640846e1278c0bb1 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Fri, 20 Oct 2023 12:13:13 -0400 Subject: [PATCH 13/54] Update to use ATP v3 This means new error handling, so multiple errors can be reported --- cmd/run-plugin/run.go | 19 ++-- internal/step/dummy/provider.go | 10 ++- internal/step/dummy/provider_test.go | 4 +- internal/step/foreach/provider.go | 12 ++- internal/step/plugin/provider.go | 79 +++++++++++------ internal/step/plugin/provider_test.go | 77 ++++++++++++++-- internal/step/provider.go | 3 + workflow/workflow.go | 121 ++++++++++++++++++++------ 8 files changed, 253 insertions(+), 72 deletions(-) diff --git a/cmd/run-plugin/run.go b/cmd/run-plugin/run.go index b75537a9..f3a19138 100644 --- a/cmd/run-plugin/run.go +++ b/cmd/run-plugin/run.go @@ -28,6 +28,7 @@ func main() { var d deployer.AnyConnectorFactory var defaultConfig any var deployerID = "docker" + var runID = "run" var workingDir string flag.StringVar(&image, "image", image, "Docker image to run") @@ -102,6 +103,7 @@ func main() { case <-ctrlC: logger.Infof("Received CTRL-C. Sending cancel signal...") toStepSignals <- schema.Input{ + RunID: runID, ID: plugin.CancellationSignalSchema.ID(), InputData: make(map[string]any), } @@ -147,19 +149,22 @@ func main() { panic(err) } fmt.Printf("Running step %s\n", stepID) - outputID, outputData, err := atpClient.Execute( - schema.Input{ID: stepID, InputData: input}, + result := atpClient.Execute( + schema.Input{RunID: runID, ID: stepID, InputData: input}, toStepSignals, nil, ) + if err := atpClient.Close(); err != nil { + fmt.Printf("Error closing ATP client: %s", err) + } output := map[string]any{ - "outputID": outputID, - "outputData": outputData, - "err": err, + "outputID": result.OutputID, + "outputData": result.OutputData, + "err": result.Error, } - result, err := yaml.Marshal(output) + resultStr, err := yaml.Marshal(output) if err != nil { panic(err) } - fmt.Printf("%s", result) + fmt.Printf("%s", resultStr) } diff --git a/internal/step/dummy/provider.go b/internal/step/dummy/provider.go index c1b06f40..e680edcb 100644 --- a/internal/step/dummy/provider.go +++ b/internal/step/dummy/provider.go @@ -201,6 +201,7 @@ type runningStep struct { stageChangeHandler step.StageChangeHandler ctx context.Context cancel context.CancelFunc + wg sync.WaitGroup lock *sync.Mutex name chan string state step.RunningStepState @@ -252,7 +253,11 @@ func (r *runningStep) Close() error { } func (r *runningStep) run() { - defer close(r.name) + r.wg.Add(1) + defer func() { + close(r.name) + r.wg.Done() + }() waitingForInput := false r.lock.Lock() if !r.inputAvailable { @@ -269,6 +274,7 @@ func (r *runningStep) run() { nil, string(StageIDGreet), waitingForInput, + &r.wg, ) select { case name, ok := <-r.name: @@ -285,7 +291,7 @@ func (r *runningStep) run() { r.lock.Lock() r.state = step.RunningStepStateFinished r.lock.Unlock() - r.stageChangeHandler.OnStepComplete(r, string(StageIDGreet), outputID, outputData) + r.stageChangeHandler.OnStepComplete(r, string(StageIDGreet), outputID, outputData, &r.wg) case <-r.ctx.Done(): return } diff --git a/internal/step/dummy/provider_test.go b/internal/step/dummy/provider_test.go index d43fe4ff..622896e4 100644 --- a/internal/step/dummy/provider_test.go +++ b/internal/step/dummy/provider_test.go @@ -2,6 +2,7 @@ package dummy_test import ( "fmt" + "sync" "testing" "go.arcalot.io/assert" @@ -13,7 +14,7 @@ type stageChangeHandler struct { message chan string } -func (s *stageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool) { +func (s *stageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool, _ *sync.WaitGroup) { } @@ -22,6 +23,7 @@ func (s *stageChangeHandler) OnStepComplete( previousStage string, previousStageOutputID *string, previousStageOutput *any, + _ *sync.WaitGroup, ) { if previousStage != "greet" { panic(fmt.Errorf("invalid previous stage: %s", previousStage)) diff --git a/internal/step/foreach/provider.go b/internal/step/foreach/provider.go index f8aaa71f..0bbd430c 100644 --- a/internal/step/foreach/provider.go +++ b/internal/step/foreach/provider.go @@ -347,6 +347,7 @@ type runningStep struct { inputAvailable bool inputData chan []any ctx context.Context + wg sync.WaitGroup cancel context.CancelFunc stageChangeHandler step.StageChangeHandler parallelism int64 @@ -406,11 +407,16 @@ func (r *runningStep) State() step.RunningStepState { func (r *runningStep) Close() error { r.cancel() + r.wg.Wait() return nil } func (r *runningStep) run() { - defer close(r.inputData) + r.wg.Add(1) + defer func() { + close(r.inputData) + r.wg.Done() + }() waitingForInput := false r.lock.Lock() if !r.inputAvailable { @@ -427,6 +433,7 @@ func (r *runningStep) run() { nil, string(StageIDExecute), waitingForInput, + &r.wg, ) select { case loopData, ok := <-r.inputData: @@ -508,12 +515,13 @@ func (r *runningStep) run() { nil, string(currentStage), false, + &r.wg, ) r.lock.Lock() r.currentState = step.RunningStepStateFinished previousStage = string(r.currentStage) r.lock.Unlock() - r.stageChangeHandler.OnStepComplete(r, previousStage, &outputID, &outputData) + r.stageChangeHandler.OnStepComplete(r, previousStage, &outputID, &outputData, &r.wg) case <-r.ctx.Done(): return } diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 9ec7c93e..5a1b4cc0 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -205,6 +205,10 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) cancel() return nil, fmt.Errorf("failed to read plugin schema from %s (%w)", image, err) } + // Tell the server that the client is done + if err := transport.Close(); err != nil { + return nil, fmt.Errorf("failed to instruct client to shut down image %s (%w)", image, err) + } // Shut down the plugin. if err := plugin.Close(); err != nil { return nil, fmt.Errorf("failed to shut down local plugin from %s (%w)", image, err) @@ -474,7 +478,7 @@ func (r *runnableStep) Start(input map[string]any, runID string, stageChangeHand pluginStepID: stepID, state: step.RunningStepStateStarting, localDeployer: r.localDeployer, - executionChannel: make(chan executionResult), + executionChannel: make(chan atp.ExecutionResult), signalToStep: make(chan schema.Input), signalFromStep: make(chan schema.Input), runID: runID, @@ -493,6 +497,7 @@ type runningStep struct { wg sync.WaitGroup ctx context.Context cancel context.CancelFunc + atpClient atp.Client deployInput chan any deployInputAvailable bool runInput chan any @@ -506,9 +511,10 @@ type runningStep struct { useLocalDeployer bool localDeployer deployer.Connector container deployer.Plugin - executionChannel chan executionResult + executionChannel chan atp.ExecutionResult signalToStep chan schema.Input // Communicates with the ATP client, not other steps. signalFromStep chan schema.Input // Communicates with the ATP client, not other steps. + closed bool // Store channels for sending pre-calculated signal outputs to other steps? // Store channels for receiving pre-calculated signal inputs from other steps? } @@ -623,7 +629,7 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro // Canceling the context should be enough when the stage isn't running. if r.currentStage == StageIDRunning { // Validated. Now call the signal. - r.signalToStep <- schema.Input{ID: cancelSignal.ID(), InputData: map[any]any{}} + r.signalToStep <- schema.Input{RunID: r.runID, ID: cancelSignal.ID(), InputData: map[any]any{}} } } // Now cancel the context to stop @@ -649,15 +655,26 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro func (r *runningStep) Close() error { r.cancel() r.lock.Lock() + if r.closed { + return nil // Already closed + } + var atpErr error + var containerErr error + if r.atpClient != nil { + atpErr = r.atpClient.Close() + } if r.container != nil { - if err := r.container.Close(); err != nil { - return fmt.Errorf("failed to stop container (%w)", err) - } + containerErr = r.container.Close() } r.container = nil r.lock.Unlock() + if containerErr != nil || atpErr != nil { + return fmt.Errorf("failed to stop atp client (%w) or container (%w)", atpErr, containerErr) + // Do not wait in this case. It may never get resolved. + } // Wait for the run to finish to ensure that it's not running after closing. r.wg.Wait() + r.closed = true return nil } @@ -714,6 +731,7 @@ func (r *runningStep) deployStage() (deployer.Plugin, error) { nil, string(StageIDDeploy), deployInputAvailable, + &r.wg, ) var deployerConfig any @@ -757,12 +775,6 @@ func (r *runningStep) deployStage() (deployer.Plugin, error) { return container, nil } -type executionResult struct { - outputID string - outputData any - err error -} - func (r *runningStep) startStage(container deployer.Plugin) error { r.logger.Debugf("Starting stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() @@ -798,6 +810,7 @@ func (r *runningStep) startStage(container deployer.Plugin) error { nil, string(StageIDStarting), runInputAvailable, + &r.wg, ) r.lock.Lock() @@ -826,9 +839,9 @@ func (r *runningStep) startStage(container deployer.Plugin) error { return fmt.Errorf("step closed while waiting for run configuration") } } - atpClient := atp.NewClientWithLogger(container, r.logger) + r.atpClient = atp.NewClientWithLogger(container, r.logger) - inputSchema, err := atpClient.ReadSchema() + inputSchema, err := r.atpClient.ReadSchema() if err != nil { return err } @@ -845,13 +858,15 @@ func (r *runningStep) startStage(container deployer.Plugin) error { // Runs the ATP client in a goroutine in order to wait for it. // On context done, the deployer has 30 seconds before it will error out. go func() { - outputID, outputData, err := atpClient.Execute( - schema.Input{ID: r.pluginStepID, InputData: runInput}, + result := r.atpClient.Execute( + schema.Input{RunID: r.runID, ID: r.pluginStepID, InputData: runInput}, r.signalToStep, r.signalFromStep, ) - r.executionChannel <- executionResult{outputID, outputData, err} - + r.executionChannel <- result + if err = r.atpClient.Close(); err != nil { + r.logger.Warningf("Error while closing ATP client: %s", err) + } }() return nil } @@ -871,13 +886,14 @@ func (r *runningStep) runStage() error { nil, string(StageIDRunning), false, + &r.wg, ) - var result executionResult + var result atp.ExecutionResult select { case result = <-r.executionChannel: - if result.err != nil { - return result.err + if result.Error != nil { + return result.Error } case <-r.ctx.Done(): // In this case, it is being instructed to stop. A signal should have been sent. @@ -913,7 +929,9 @@ func (r *runningStep) runStage() error { nil, nil, string(r.currentStage), - false) + false, + &r.wg, + ) r.lock.Lock() r.state = step.RunningStepStateFinished @@ -921,8 +939,9 @@ func (r *runningStep) runStage() error { r.stageChangeHandler.OnStepComplete( r, string(r.currentStage), - &result.outputID, - &result.outputData, + &result.OutputID, + &result.OutputData, + &r.wg, ) return nil @@ -945,6 +964,7 @@ func (r *runningStep) deployFailed(err error) { nil, string(StageIDDeployFailed), false, + &r.wg, ) r.logger.Warningf("Plugin step %s/%s deploy failed. %v", r.runID, r.pluginStepID, err) @@ -963,6 +983,7 @@ func (r *runningStep) deployFailed(err error) { string(r.currentStage), &outputID, &output, + &r.wg, ) } @@ -979,7 +1000,9 @@ func (r *runningStep) startFailed(err error) { nil, nil, string(r.currentStage), - false) + false, + &r.wg, + ) r.logger.Warningf("Plugin step %s/%s start failed. %v", r.runID, r.pluginStepID, err) // Now it's done. @@ -997,6 +1020,7 @@ func (r *runningStep) startFailed(err error) { string(r.currentStage), &outputID, &output, + &r.wg, ) } @@ -1018,7 +1042,9 @@ func (r *runningStep) runFailed(err error) { nil, nil, string(r.currentStage), - false) + false, + &r.wg, + ) r.logger.Warningf("Plugin step %s/%s run failed. %v", r.runID, r.pluginStepID, err) @@ -1037,5 +1063,6 @@ func (r *runningStep) runFailed(err error) { string(r.currentStage), &outputID, &output, + &r.wg, ) } diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 7e605672..2db5f39b 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -9,6 +9,7 @@ import ( "go.flow.arcalot.io/engine/internal/step" "go.flow.arcalot.io/engine/internal/step/plugin" testdeployer "go.flow.arcalot.io/testdeployer" + "sync" "testing" ) @@ -16,7 +17,7 @@ type deployFailStageChangeHandler struct { message chan string } -func (s *deployFailStageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool) { +func (s *deployFailStageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool, _ *sync.WaitGroup) { } @@ -25,6 +26,7 @@ func (s *deployFailStageChangeHandler) OnStepComplete( previousStage string, previousStageOutputID *string, previousStageOutput *any, + _ *sync.WaitGroup, ) { if previousStage != string(plugin.StageIDDeployFailed) { panic(fmt.Errorf("invalid previous stage: %s", previousStage)) @@ -47,7 +49,7 @@ type startFailStageChangeHandler struct { message chan string } -func (s *startFailStageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool) { +func (s *startFailStageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool, _ *sync.WaitGroup) { } @@ -56,6 +58,7 @@ func (s *startFailStageChangeHandler) OnStepComplete( previousStage string, previousStageOutputID *string, previousStageOutput *any, + _ *sync.WaitGroup, ) { if previousStage != string(plugin.StageIDCrashed) { panic(fmt.Errorf("invalid previous stage: %s", previousStage)) @@ -79,7 +82,7 @@ type stageChangeHandler struct { message chan string } -func (s *stageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool) { +func (s *stageChangeHandler) OnStageChange(_ step.RunningStep, _ *string, _ *string, _ *any, _ string, _ bool, _ *sync.WaitGroup) { } @@ -88,6 +91,7 @@ func (s *stageChangeHandler) OnStepComplete( previousStage string, previousStageOutputID *string, previousStageOutput *any, + _ *sync.WaitGroup, ) { if previousStage != string(plugin.StageIDOutput) { panic(fmt.Errorf("invalid previous stage: %s", previousStage)) @@ -141,8 +145,12 @@ func TestProvider_Utility(t *testing.T) { _, err = runnable.Lifecycle(map[string]any{"step": "wait"}) assert.NoError(t, err) - _, err = runnable.Lifecycle(map[string]any{"step": nil}) + _, err = runnable.Lifecycle(map[string]any{"step": "hello"}) assert.NoError(t, err) + + // There is more than one step, so no specified one will cause an error. + _, err = runnable.Lifecycle(map[string]any{"step": nil}) + assert.Error(t, err) } func TestProvider_HappyError(t *testing.T) { @@ -185,8 +193,8 @@ func TestProvider_HappyError(t *testing.T) { _, err = runnable.Start(map[string]any{"step": "wrong_stepid"}, t.Name(), handler) assert.Error(t, err) - // default step id - running, err := runnable.Start(map[string]any{"step": nil}, t.Name(), handler) + // wait step + running, err := runnable.Start(map[string]any{"step": "wait"}, t.Name(), handler) assert.NoError(t, err) // non-existent stage @@ -255,6 +263,59 @@ func TestProvider_HappyError(t *testing.T) { }) } +func TestProvider_VerifyCancelSignal(t *testing.T) { + logger := log.New( + log.Config{ + Level: log.LevelError, + Destination: log.DestinationStdout, + }, + ) + workflowDeployerCfg := map[string]any{ + "type": "test-impl", + } + + deployerRegistry := deployer_registry.New( + deployer.Any(testdeployer.NewFactory())) + + plp, err := plugin.New( + logger, + deployerRegistry, + workflowDeployerCfg, + ) + assert.NoError(t, err) + + runnable, err := plp.LoadSchema( + map[string]any{"plugin": "simulation"}, map[string][]byte{}) + assert.NoError(t, err) + assert.NotNil(t, runnable) + + waitLifecycle, err := runnable.Lifecycle(map[string]any{"step": "wait"}) + assert.NoError(t, err) + // Verify that the expected lifecycle stage is there, then verify that cancel is disabled. + waitCancelledStageIDIndex := assert.SliceContainsExtractor(t, + func(schema step.LifecycleStageWithSchema) string { + return schema.ID + }, string(plugin.StageIDCancelled), waitLifecycle.Stages) + waitStageIDCancelled := waitLifecycle.Stages[waitCancelledStageIDIndex] + waitStopIfSchema := assert.MapContainsKey(t, "stop_if", waitStageIDCancelled.InputSchema) + if waitStopIfSchema.Disabled { + t.Fatalf("step wait's wait_for schema is disabled when the cancel signal is present.") + } + + helloLifecycle, err := runnable.Lifecycle(map[string]any{"step": "hello"}) + assert.NoError(t, err) + // Verify that the expected lifecycle stage is there, then verify that cancel is disabled. + helloCancelledStageIDIndex := assert.SliceContainsExtractor(t, + func(schema step.LifecycleStageWithSchema) string { + return schema.ID + }, string(plugin.StageIDCancelled), helloLifecycle.Stages) + helloStageIDCancelled := helloLifecycle.Stages[helloCancelledStageIDIndex] + helloStopIfSchema := assert.MapContainsKey(t, "stop_if", helloStageIDCancelled.InputSchema) + if !helloStopIfSchema.Disabled { + t.Fatalf("step hello's stop_if schema is not disabled when the cancel signal is not present.") + } +} + func TestProvider_DeployFail(t *testing.T) { logConfig := log.Config{ Level: log.LevelError, @@ -289,8 +350,8 @@ func TestProvider_DeployFail(t *testing.T) { message: make(chan string), } - // default step id - running, err := runnable.Start(map[string]any{"step": nil}, t.Name(), handler) + // wait step + running, err := runnable.Start(map[string]any{"step": "wait"}, t.Name(), handler) assert.NoError(t, err) assert.NoError(t, running.ProvideStageInput( diff --git a/internal/step/provider.go b/internal/step/provider.go index aa252116..203ff152 100644 --- a/internal/step/provider.go +++ b/internal/step/provider.go @@ -2,6 +2,7 @@ package step import ( "go.flow.arcalot.io/pluginsdk/schema" + "sync" ) // Provider is the description of an item that fits in a workflow. Its implementation provide the @@ -43,6 +44,7 @@ type StageChangeHandler interface { previousStageOutput *any, newStage string, inputAvailable bool, + wg *sync.WaitGroup, ) // OnStepComplete is called when the step has completed a final stage in its lifecycle and communicates the output. @@ -52,6 +54,7 @@ type StageChangeHandler interface { previousStage string, previousStageOutputID *string, previousStageOutput *any, + wg *sync.WaitGroup, ) } diff --git a/workflow/workflow.go b/workflow/workflow.go index 0d107926..577af7e0 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -74,6 +74,7 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s outputDone: false, cancel: cancel, workflowContext: e.workflowContext, + recentErrors: make(chan error, 20), // Big buffer in case there are lots of subsequent errors. } l.lock.Lock() @@ -95,22 +96,36 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s l.data["steps"].(map[string]any)[stepID] = stepDataModel var stageHandler step.StageChangeHandler = &stageChangeHandler{ - onStageChange: func(step step.RunningStep, previousStage *string, previousStageOutputID *string, previousStageOutput *any, stage string, inputAvailable bool) { + onStageChange: func( + step step.RunningStep, + previousStage *string, + previousStageOutputID *string, + previousStageOutput *any, + stage string, + inputAvailable bool, + wg *sync.WaitGroup, + ) { waitingForInputText := "" if !inputAvailable { waitingForInputText = " and is waiting for input" } e.logger.Debugf("START Stage change for step %s to %s%s...", stepID, stage, waitingForInputText) - l.onStageComplete(stepID, previousStage, previousStageOutputID, previousStageOutput) + l.onStageComplete(stepID, previousStage, previousStageOutputID, previousStageOutput, wg) e.logger.Debugf("DONE Stage change for step %s to %s%s...", stepID, stage, waitingForInputText) }, - onStepComplete: func(step step.RunningStep, previousStage string, previousStageOutputID *string, previousStageOutput *any) { + onStepComplete: func( + step step.RunningStep, + previousStage string, + previousStageOutputID *string, + previousStageOutput *any, + wg *sync.WaitGroup, + ) { if previousStageOutputID != nil { e.logger.Debugf("Step %s completed with stage '%s', output '%s'...", stepID, previousStage, *previousStageOutputID) } else { e.logger.Debugf("Step %s completed with stage '%s'...", stepID, previousStage) } - l.onStageComplete(stepID, &previousStage, previousStageOutputID, previousStageOutput) + l.onStageComplete(stepID, &previousStage, previousStageOutputID, previousStageOutput, wg) }, } e.logger.Debugf("Launching step %s...", stepID) @@ -176,9 +191,10 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s } return outputDataEntry.outputID, outputData, nil case <-ctx.Done(): - e.logger.Debugf("Workflow execution aborted. %s", l.lastError) - if l.lastError != nil { - return "", nil, l.lastError + lastErr := l.getLastError() + e.logger.Debugf("Workflow execution aborted. %s", lastErr) + if lastErr != nil { + return "", nil, lastErr } return "", nil, fmt.Errorf("workflow execution aborted (%w)", ctx.Err()) } @@ -202,16 +218,39 @@ type loopState struct { runningSteps map[string]step.RunningStep outputDataChannel chan outputDataType outputDone bool - lastError error + recentErrors chan error cancel context.CancelFunc workflowContext map[string][]byte } -func (l *loopState) onStageComplete(stepID string, previousStage *string, previousStageOutputID *string, previousStageOutput *any) { +// getLastError gathers the last errors. If there are several, it creates a new one that consolidates them. +// This will read from the channel. Calling again will only gather new errors since the last call. +func (l *loopState) getLastError() error { + var errors []error +errGatherLoop: + for { + select { + case err := <-l.recentErrors: + errors = append(errors, err) + default: + break errGatherLoop // No more errors + } + } + switch len(errors) { + case 0: + return nil + case 1: + return errors[0] + default: + return fmt.Errorf("multiple errors: %v", errors) + } +} + +func (l *loopState) onStageComplete(stepID string, previousStage *string, previousStageOutputID *string, previousStageOutput *any, wg *sync.WaitGroup) { l.lock.Lock() defer func() { if previousStage != nil { - l.checkForDeadlocks(3) + l.checkForDeadlocks(3, wg) } l.lock.Unlock() }() @@ -222,14 +261,14 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo stageNode, err := l.dag.GetNodeByID(GetStageNodeID(stepID, *previousStage)) if err != nil { l.logger.Errorf("Failed to get stage node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) - l.lastError = fmt.Errorf("failed to get stage node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.recentErrors <- fmt.Errorf("failed to get stage node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) l.cancel() return } l.logger.Debugf("Removed node '%s' from the DAG", stageNode.ID()) if err := stageNode.Remove(); err != nil { l.logger.Errorf("Failed to remove stage node ID %s (%w)", stageNode.ID(), err) - l.lastError = fmt.Errorf("failed to remove stage node ID %s (%w)", stageNode.ID(), err) + l.recentErrors <- fmt.Errorf("failed to remove stage node ID %s (%w)", stageNode.ID(), err) l.cancel() return } @@ -237,7 +276,7 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo outputNode, err := l.dag.GetNodeByID(GetOutputNodeID(stepID, *previousStage, *previousStageOutputID)) if err != nil { l.logger.Errorf("Failed to get output node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) - l.lastError = fmt.Errorf("failed to get output node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.recentErrors <- fmt.Errorf("failed to get output node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) l.cancel() return } @@ -245,7 +284,7 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo l.logger.Debugf("Removed node '%s' from the DAG", outputNode.ID()) if err := outputNode.Remove(); err != nil { l.logger.Errorf("Failed to remove output node ID %s (%w)", outputNode.ID(), err) - l.lastError = fmt.Errorf("failed to remove output node ID %s (%w)", outputNode.ID(), err) + l.recentErrors <- fmt.Errorf("failed to remove output node ID %s (%w)", outputNode.ID(), err) l.cancel() return } @@ -315,7 +354,7 @@ func (l *loopState) notifySteps() { //nolint:gocognit // Tries to match the schema if _, err := node.Item().DataSchema.Unserialize(untypedInputData); err != nil { l.logger.Errorf("Bug: schema evaluation resulted in invalid data for %s (%v)", node.ID(), err) - l.lastError = fmt.Errorf("bug: schema evaluation resulted in invalid data for %s (%w)", node.ID(), err) + l.recentErrors <- fmt.Errorf("bug: schema evaluation resulted in invalid data for %s (%w)", node.ID(), err) l.cancel() return } @@ -338,7 +377,7 @@ func (l *loopState) notifySteps() { //nolint:gocognit typedInputData, ); err != nil { l.logger.Errorf("Bug: failed to provide input to step %s (%w)", node.Item().StepID, err) - l.lastError = fmt.Errorf("bug: failed to provide input to step %s (%w)", node.Item().StepID, err) + l.recentErrors <- fmt.Errorf("bug: failed to provide input to step %s (%w)", node.Item().StepID, err) l.cancel() return } @@ -369,7 +408,7 @@ func (l *loopState) notifySteps() { //nolint:gocognit } } -func (l *loopState) checkForDeadlocks(retries int) { +func (l *loopState) checkForDeadlocks(retries int, wg *sync.WaitGroup) { // Here we make sure we don't have a deadlock. counters := struct { starting int @@ -438,19 +477,20 @@ func (l *loopState) checkForDeadlocks(retries int) { ) if counters.starting == 0 && counters.running == 0 && counters.waitingWithoutInbound == 0 && !l.outputDone { if retries <= 0 { - l.lastError = &ErrNoMorePossibleSteps{ + l.recentErrors <- &ErrNoMorePossibleSteps{ l.dag, } - l.logger.Errorf("%v", l.lastError) l.logger.Debugf("DAG:\n%s", l.dag.Mermaid()) l.cancel() } else { // Retry. There are times when all the steps are in a transition state. // Retrying will delay the check until after they are done with the transition. l.logger.Warningf("No running steps. Rechecking...") + wg.Add(1) go func() { time.Sleep(5 * time.Millisecond) - l.checkForDeadlocks(retries - 1) + l.checkForDeadlocks(retries-1, wg) + wg.Done() }() } } @@ -493,15 +533,44 @@ func (l *loopState) resolveExpressions(inputData any, dataModel any) (any, error } } +// stageChangeHandler is implementing step.StageChangeHandler type stageChangeHandler struct { - onStageChange func(step step.RunningStep, previousStage *string, previousStageOutputID *string, previousStageOutput *any, stage string, waitingForInput bool) - onStepComplete func(step step.RunningStep, previousStage string, previousStageOutputID *string, previousStageOutput *any) + onStageChange func( + step step.RunningStep, + previousStage *string, + previousStageOutputID *string, + previousStageOutput *any, + stage string, + waitingForInput bool, + wg *sync.WaitGroup, + ) + onStepComplete func( + step step.RunningStep, + previousStage string, + previousStageOutputID *string, + previousStageOutput *any, + wg *sync.WaitGroup, + ) } -func (s stageChangeHandler) OnStageChange(step step.RunningStep, previousStage *string, previousStageOutputID *string, previousStageOutput *any, stage string, waitingForInput bool) { - s.onStageChange(step, previousStage, previousStageOutputID, previousStageOutput, stage, waitingForInput) +func (s stageChangeHandler) OnStageChange( + step step.RunningStep, + previousStage *string, + previousStageOutputID *string, + previousStageOutput *any, + stage string, + waitingForInput bool, + wg *sync.WaitGroup, +) { + s.onStageChange(step, previousStage, previousStageOutputID, previousStageOutput, stage, waitingForInput, wg) } -func (s stageChangeHandler) OnStepComplete(step step.RunningStep, previousStage string, previousStageOutputID *string, previousStageOutput *any) { - s.onStepComplete(step, previousStage, previousStageOutputID, previousStageOutput) +func (s stageChangeHandler) OnStepComplete( + step step.RunningStep, + previousStage string, + previousStageOutputID *string, + previousStageOutput *any, + wg *sync.WaitGroup, +) { + s.onStepComplete(step, previousStage, previousStageOutputID, previousStageOutput, wg) } From 30b9c23e0c5dbca85e8a8ec46ab40dab22fc707a Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Fri, 20 Oct 2023 12:16:09 -0400 Subject: [PATCH 14/54] Update SDK in go mod, and add force stop to step interface Force stop is good for when you do not need to wait for the ATP client, or something equivalant, to finish. Waiting for the ATP to finish means waiting for the steps to finish entirely. --- go.mod | 4 ++-- go.sum | 8 ++++---- internal/step/dummy/provider.go | 4 ++++ internal/step/foreach/provider.go | 5 +++++ internal/step/plugin/provider.go | 29 +++++++++++++++++++++++------ internal/step/provider.go | 2 ++ workflow/workflow.go | 2 +- 7 files changed, 41 insertions(+), 13 deletions(-) diff --git a/go.mod b/go.mod index 1773a7a3..7bd5a079 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module go.flow.arcalot.io/engine go 1.18 require ( - go.arcalot.io/assert v1.5.0 + go.arcalot.io/assert v1.6.0 go.arcalot.io/dgraph v1.1.0 go.arcalot.io/lang v1.0.0 go.arcalot.io/log/v2 v2.0.0 @@ -11,7 +11,7 @@ require ( go.flow.arcalot.io/dockerdeployer v0.3.0 go.flow.arcalot.io/expressions v0.2.0 go.flow.arcalot.io/kubernetesdeployer v0.5.1 - go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e + go.flow.arcalot.io/pluginsdk v0.5.0-beta.1 go.flow.arcalot.io/podmandeployer v0.3.1 go.flow.arcalot.io/pythondeployer v0.1.4 go.flow.arcalot.io/testdeployer v0.2.1-0.20230914224521-f83e94942739 diff --git a/go.sum b/go.sum index 4afc3999..44082d92 100644 --- a/go.sum +++ b/go.sum @@ -143,8 +143,8 @@ github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1: github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -go.arcalot.io/assert v1.5.0 h1:6Ifu7jBpHA6s1aEzf8nlv51MXyBoqDkLzmbvjoZpcXM= -go.arcalot.io/assert v1.5.0/go.mod h1:Xy3ScX0p9IMY89gdsgexOKxnmDr0nGHG9dV7p8Uxg7w= +go.arcalot.io/assert v1.6.0 h1:iKA8SZZ1MRblMX5QAwwY5RbpR+VNyp//4IU7vo08Xu0= +go.arcalot.io/assert v1.6.0/go.mod h1:Xy3ScX0p9IMY89gdsgexOKxnmDr0nGHG9dV7p8Uxg7w= go.arcalot.io/dgraph v1.1.0 h1:c0LR7+xdUy7Ki6e4nR9rBvK0Upr4Nu49fu+poP/9WMg= go.arcalot.io/dgraph v1.1.0/go.mod h1:FuNv92OgHsJYepD6Unwn+S/4DioBnv06JxQ2BtQct7E= go.arcalot.io/lang v1.0.0 h1:mgDaieT4wWdZTnR4V7+/pgYRmzfU7VZZgIzHccuxAbY= @@ -159,8 +159,8 @@ go.flow.arcalot.io/expressions v0.2.0 h1:2k8InnpLqVmv5SDvJ1xRz1ubqF+zKN44U08D50T go.flow.arcalot.io/expressions v0.2.0/go.mod h1:m4p6oCwjjxRjPCAYGeZDNP9DGfoDODHo1z4pBoO+Lpc= go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEXBAnyoWJnHls= go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= -go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e h1:Wh4RQdU9x1GNL8e0gppx9kKmjKC05QHt3eA8rssp2S0= -go.flow.arcalot.io/pluginsdk v0.4.0-beta.1.0.20230914222256-af7a31f84b1e/go.mod h1:8Gp8Q0Eo0s+2I4nNl3uQ95jPi2zFKEX7JqyjhmK7BiI= +go.flow.arcalot.io/pluginsdk v0.5.0-beta.1 h1:60n+MXI7GtRFFUfrQN8OD+R1BraVcGAx0cxcnruGnJk= +go.flow.arcalot.io/pluginsdk v0.5.0-beta.1/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= go.flow.arcalot.io/podmandeployer v0.3.1/go.mod h1:SmROc9nHG+KfKasyTeKtGmI9EBlXCupXjBIgX5glGn8= go.flow.arcalot.io/pythondeployer v0.1.4 h1:A4YXgfzrzJZLp19kK8PV5B+ZG6MieiYSPILZSqDMUqE= diff --git a/internal/step/dummy/provider.go b/internal/step/dummy/provider.go index e680edcb..9df7f8f9 100644 --- a/internal/step/dummy/provider.go +++ b/internal/step/dummy/provider.go @@ -252,6 +252,10 @@ func (r *runningStep) Close() error { return nil } +func (r *runningStep) ForceClose() error { + return r.Close() +} + func (r *runningStep) run() { r.wg.Add(1) defer func() { diff --git a/internal/step/foreach/provider.go b/internal/step/foreach/provider.go index 0bbd430c..55d3b58b 100644 --- a/internal/step/foreach/provider.go +++ b/internal/step/foreach/provider.go @@ -411,6 +411,11 @@ func (r *runningStep) Close() error { return nil } +func (r *runningStep) ForceClose() error { + // For now, unless it becomes a problem, we'll just call the normal close function. + return r.Close() +} + func (r *runningStep) run() { r.wg.Add(1) defer func() { diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 5a1b4cc0..b118903f 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -192,6 +192,7 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) ctx, cancel := context.WithCancel(context.Background()) defer cancel() + plugin, err := p.localDeployer.Deploy(ctx, image) if err != nil { cancel() @@ -652,7 +653,26 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro } } +// ForceClose closes the step without waiting for a graceful shutdown of the ATP client +// Warning: This means that +func (r *runningStep) ForceClose() error { + err := r.closeComponents(false) + // Wait for the run to finish to ensure that it's not running after closing. + r.wg.Wait() + r.closed = true + r.logger.Warningf("Step %s/%s force closed.", r.runID, r.pluginStepID) + return err +} + func (r *runningStep) Close() error { + err := r.closeComponents(true) + // Wait for the run to finish to ensure that it's not running after closing. + r.wg.Wait() + r.closed = true + return err +} + +func (r *runningStep) closeComponents(closeATP bool) error { r.cancel() r.lock.Lock() if r.closed { @@ -660,7 +680,7 @@ func (r *runningStep) Close() error { } var atpErr error var containerErr error - if r.atpClient != nil { + if r.atpClient != nil && closeATP { atpErr = r.atpClient.Close() } if r.container != nil { @@ -672,9 +692,6 @@ func (r *runningStep) Close() error { return fmt.Errorf("failed to stop atp client (%w) or container (%w)", atpErr, containerErr) // Do not wait in this case. It may never get resolved. } - // Wait for the run to finish to ensure that it's not running after closing. - r.wg.Wait() - r.closed = true return nil } @@ -903,9 +920,9 @@ func (r *runningStep) runStage() error { case result = <-r.executionChannel: // Successfully stopped before end of timeout. case <-time.After(time.Duration(30) * time.Second): - r.logger.Warningf("Step %s/%s did not terminate within the 30 second time limit. Closing container.", + r.logger.Warningf("Step %s/%s did not complete within the 30 second time limit. Force closing container.", r.runID, r.pluginStepID) - if err := r.Close(); err != nil { + if err := r.ForceClose(); err != nil { r.logger.Warningf("Error in step %s/%s while closing plugin container (%w)", r.runID, r.pluginStepID, err) } } diff --git a/internal/step/provider.go b/internal/step/provider.go index 203ff152..b8722486 100644 --- a/internal/step/provider.go +++ b/internal/step/provider.go @@ -101,4 +101,6 @@ type RunningStep interface { State() RunningStepState // Close shuts down the step and cleans up the resources associated with the step. Close() error + // ForceClose shuts down the step forcefully. + ForceClose() error } diff --git a/workflow/workflow.go b/workflow/workflow.go index 577af7e0..a51510ee 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -141,7 +141,7 @@ func (e *executableWorkflow) Execute(ctx context.Context, input any) (outputID s e.logger.Debugf("Terminating all steps...") for stepID, runningStep := range l.runningSteps { e.logger.Debugf("Terminating step %s...", stepID) - if err := runningStep.Close(); err != nil { + if err := runningStep.ForceClose(); err != nil { panic(fmt.Errorf("failed to close step %s (%w)", stepID, err)) } } From 4f87b195cb79874f970c1a076b53c6c694e1f747 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Fri, 20 Oct 2023 12:49:19 -0400 Subject: [PATCH 15/54] Cleanup --- internal/step/lifecycle.go | 2 -- internal/step/plugin/provider.go | 15 +-------------- 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/internal/step/lifecycle.go b/internal/step/lifecycle.go index e7ad97e3..83495250 100644 --- a/internal/step/lifecycle.go +++ b/internal/step/lifecycle.go @@ -73,8 +73,6 @@ type LifecycleStage struct { // It will automatically create a DAG node between the current and the described next stages to ensure // that it is running in order. NextStages []string - // RemovedStages describes stages that should be removed from the DAG if this stage is run. - RemovedStages []string // Fatal indicates that this stage should be treated as fatal unless handled by the workflow. Fatal bool } diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index b118903f..a8316fc7 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -569,7 +569,6 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro r.state = step.RunningStepStateRunning } - // TODO: CHECK IF THIS IS OKAY // Feed the deploy step its input. select { case r.deployInput <- unserializedDeployerConfig: @@ -596,14 +595,10 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro } // Make sure we transition the state before unlocking so there are no race conditions. r.runInputAvailable = true - //if r.state == step.RunningStepStateWaitingForInput && r.currentStage == StageIDDeploy { - // r.logger.Debugf("Input available. State set to running.") - // r.state = step.RunningStepStateRunning - //} + // Unlock before passing the data over the channel to prevent a deadlock. // The other end of the channel needs to be unlocked to read the data. - // TODO: CHECK IF THIS WORKS // Feed the run step its input over the channel. select { case r.runInput <- input["input"]: @@ -809,14 +804,6 @@ func (r *runningStep) startStage(container deployer.Plugin) error { r.state = step.RunningStepStateWaitingForInput } - //if !r.runInputAvailable { - // r.logger.Debugf("Waiting for input state while starting 1.") - // //r.state = step.RunningStepStateWaitingForInput - // // TEMP: Assume running until stage change - // r.state = step.RunningStepStateRunning - //} else { - // r.state = step.RunningStepStateRunning - //} runInputAvailable := r.runInputAvailable r.lock.Unlock() From e1768b58f3610b2ecda8ca72f575633d116ac4e8 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Fri, 20 Oct 2023 13:06:58 -0400 Subject: [PATCH 16/54] Fix tests --- workflow/executor_test.go | 4 ++++ workflow/workflow_test.go | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/workflow/executor_test.go b/workflow/executor_test.go index 77658b64..80fae2c4 100644 --- a/workflow/executor_test.go +++ b/workflow/executor_test.go @@ -115,6 +115,7 @@ func TestSharedInput(t *testing.T) { } var missingInputWorkflowDefinition1 = ` +version: v0.1.0 input: root: RootObject objects: @@ -132,6 +133,7 @@ outputs: ` var missingInputWorkflowDefinition2 = ` +version: v0.1.0 input: root: RootObject objects: @@ -157,6 +159,7 @@ func TestMissingInput(t *testing.T) { } var mismatchedStepInputTypesWorkflowDefinition = ` +version: v0.1.0 input: root: RootObject objects: @@ -186,6 +189,7 @@ func TestMismatchedStepInputTypes(t *testing.T) { } var mismatchedInputTypesWorkflowDefinition = ` +version: v0.1.0 input: root: RootObject objects: diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index 1b6b7acf..43ba7467 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -90,6 +90,7 @@ func TestStepCancellation(t *testing.T) { } var earlyStepCancellationWorkflowDefinition = ` +version: v0.1.0 input: root: RootObject objects: @@ -155,6 +156,7 @@ func TestEarlyStepCancellation(t *testing.T) { } var deploymentStepCancellationWorkflowDefinition = ` +version: v0.1.0 input: root: RootObject objects: @@ -216,6 +218,7 @@ func TestDeploymentStepCancellation(t *testing.T) { } var simpleValidLiteralInputWaitWorkflowDefinition = ` +version: v0.1.0 input: root: RootObject objects: @@ -399,6 +402,7 @@ func TestWaitForParallel(t *testing.T) { } var missingInputsFailedDeploymentWorkflowDefinition = ` +version: v0.1.0 input: root: RootObject objects: @@ -437,6 +441,7 @@ func TestMissingInputsFailedDeployment(t *testing.T) { } var missingInputsWrongOutputWorkflowDefinition = ` +version: v0.1.0 input: root: RootObject objects: From f1657777744daaca4433756d24f713e081533fd2 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Tue, 24 Oct 2023 11:43:10 -0400 Subject: [PATCH 17/54] Update SDK and test deployer --- go.mod | 6 +++--- go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/go.mod b/go.mod index 44becfe3..20fd4c59 100644 --- a/go.mod +++ b/go.mod @@ -11,10 +11,10 @@ require ( go.flow.arcalot.io/dockerdeployer v0.3.0 go.flow.arcalot.io/expressions v0.2.0 go.flow.arcalot.io/kubernetesdeployer v0.5.1 - go.flow.arcalot.io/pluginsdk v0.5.0-beta.1 + go.flow.arcalot.io/pluginsdk v0.5.0-beta.2 go.flow.arcalot.io/podmandeployer v0.3.1 go.flow.arcalot.io/pythondeployer v0.2.0 - go.flow.arcalot.io/testdeployer v0.2.1-0.20230914224521-f83e94942739 + go.flow.arcalot.io/testdeployer v0.3.0-beta.1 gopkg.in/yaml.v3 v3.0.1 ) @@ -48,7 +48,7 @@ require ( github.com/opencontainers/image-spec v1.0.2 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/x448/float16 v0.8.4 // indirect - go.flow.arcalot.io/testplugin v0.1.1-0.20230914224236-656dd83abb0e // indirect + go.flow.arcalot.io/testplugin v0.2.1 // indirect golang.org/x/mod v0.13.0 // indirect golang.org/x/net v0.15.0 // indirect golang.org/x/oauth2 v0.2.0 // indirect diff --git a/go.sum b/go.sum index bef4c454..f64f0bce 100644 --- a/go.sum +++ b/go.sum @@ -164,16 +164,16 @@ go.flow.arcalot.io/expressions v0.2.0 h1:2k8InnpLqVmv5SDvJ1xRz1ubqF+zKN44U08D50T go.flow.arcalot.io/expressions v0.2.0/go.mod h1:m4p6oCwjjxRjPCAYGeZDNP9DGfoDODHo1z4pBoO+Lpc= go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEXBAnyoWJnHls= go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= -go.flow.arcalot.io/pluginsdk v0.5.0-beta.1 h1:60n+MXI7GtRFFUfrQN8OD+R1BraVcGAx0cxcnruGnJk= -go.flow.arcalot.io/pluginsdk v0.5.0-beta.1/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= +go.flow.arcalot.io/pluginsdk v0.5.0-beta.2 h1:y69ZKZKghAV4cb3w2XPGFculRo+3mWxwJXX2WPfw0qQ= +go.flow.arcalot.io/pluginsdk v0.5.0-beta.2/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= go.flow.arcalot.io/podmandeployer v0.3.1/go.mod h1:SmROc9nHG+KfKasyTeKtGmI9EBlXCupXjBIgX5glGn8= go.flow.arcalot.io/pythondeployer v0.2.0 h1:iSVxQzRGrEr0/bJfsRNQ7Q3ItA3sr0+7jDg1SL15llo= go.flow.arcalot.io/pythondeployer v0.2.0/go.mod h1:zegeDjxiddprrPmO6243AslQX7BxCmV46xXSUG5TZGc= -go.flow.arcalot.io/testdeployer v0.2.1-0.20230914224521-f83e94942739 h1:uGz0wU4FXB3Llfk8mWsVP+BQGMTE2LJdfFOaRDxFfuo= -go.flow.arcalot.io/testdeployer v0.2.1-0.20230914224521-f83e94942739/go.mod h1:6oX0Kx7zwMxEdB/6h7LoIPiTTYgVYS3FJ0MdqlH6i6k= -go.flow.arcalot.io/testplugin v0.1.1-0.20230914224236-656dd83abb0e h1:xb0swZIsOerWcN2/76HnzQhJvJkL+1NUgnKw7EqhArc= -go.flow.arcalot.io/testplugin v0.1.1-0.20230914224236-656dd83abb0e/go.mod h1:HBjElWQmsQeBWPTBNFc1EHAatWFMG13CuanhCmJ/0Jw= +go.flow.arcalot.io/testdeployer v0.3.0-beta.1 h1:OgTdbOnqzmLdwG/GVnnYBxsrgqD1hySDjSm54j6Gvlo= +go.flow.arcalot.io/testdeployer v0.3.0-beta.1/go.mod h1:qTd2I5m46jR78e5ivoQ94sTNNFsJRLROScQ35+v9TNE= +go.flow.arcalot.io/testplugin v0.2.1 h1:9kQ2MKvcXtEcwk5c4qSWN+FovpER2C9vn730laAm9iE= +go.flow.arcalot.io/testplugin v0.2.1/go.mod h1:ZoVF8tIKppQmj5nvoZPA48GQ7BuoWXQcuCw2x2sJxjE= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= From d8c64bcf7b084e0aee9a88abf8250c6d450dae09 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Tue, 24 Oct 2023 17:01:11 -0400 Subject: [PATCH 18/54] Reduce redundancy in Plugin Provider --- internal/step/plugin/provider.go | 160 +++++++------------------------ 1 file changed, 35 insertions(+), 125 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index a8316fc7..0dee2557 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -726,13 +726,7 @@ func (r *runningStep) run() { func (r *runningStep) deployStage() (deployer.Plugin, error) { r.logger.Debugf("Deploying stage for step %s/%s", r.runID, r.pluginStepID) r.lock.Lock() - if !r.deployInputAvailable { - r.logger.Debugf("PROCESSING inputs state while deploying.") - //r.state = step.RunningStepStateWaitingForInput - r.state = step.RunningStepStateRunning - } else { - r.state = step.RunningStepStateRunning - } + r.state = step.RunningStepStateRunning deployInputAvailable := r.deployInputAvailable r.lock.Unlock() @@ -877,21 +871,7 @@ func (r *runningStep) startStage(container deployer.Plugin) error { func (r *runningStep) runStage() error { r.logger.Debugf("Running stage for step %s/%s", r.runID, r.pluginStepID) - r.lock.Lock() - previousStage := string(r.currentStage) - r.currentStage = StageIDRunning - r.state = step.RunningStepStateRunning - r.lock.Unlock() - - r.stageChangeHandler.OnStageChange( - r, - &previousStage, - nil, - nil, - string(StageIDRunning), - false, - &r.wg, - ) + r.transitionStage(StageIDRunning, step.RunningStepStateRunning) var result atp.ExecutionResult select { @@ -916,157 +896,87 @@ func (r *runningStep) runStage() error { } - // Execution complete, move to finished stage. - r.lock.Lock() - // Be careful that everything here is set correctly. - // Else it will cause undesired behavior. - previousStage = string(r.currentStage) - r.currentStage = StageIDOutput - // First running, then state change, then finished. - // This is so it properly steps through all the stages it needs to. - r.state = step.RunningStepStateRunning - r.lock.Unlock() - - r.stageChangeHandler.OnStageChange( - r, - &previousStage, - nil, - nil, - string(r.currentStage), - false, - &r.wg, - ) - - r.lock.Lock() - r.state = step.RunningStepStateFinished - r.lock.Unlock() - r.stageChangeHandler.OnStepComplete( - r, - string(r.currentStage), - &result.OutputID, - &result.OutputData, - &r.wg, - ) + // Execution complete, move to state running stage outputs, then to state finished stage. + r.transitionStage(StageIDOutput, step.RunningStepStateRunning) + r.completeStage(r.currentStage, step.RunningStepStateFinished, &result.OutputID, &result.OutputData) return nil } func (r *runningStep) deployFailed(err error) { r.logger.Debugf("Deploy failed stage for step %s/%s", r.runID, r.pluginStepID) - r.lock.Lock() - previousStage := string(r.currentStage) - r.currentStage = StageIDDeployFailed - // Don't forget to update this, or else it will behave very oddly. - // First running, then finished. You can't skip states. - r.state = step.RunningStepStateRunning - r.lock.Unlock() - - r.stageChangeHandler.OnStageChange( - r, - &previousStage, - nil, - nil, - string(StageIDDeployFailed), - false, - &r.wg, - ) + r.transitionStage(StageIDDeployFailed, step.RunningStepStateRunning) r.logger.Warningf("Plugin step %s/%s deploy failed. %v", r.runID, r.pluginStepID, err) // Now it's done. - r.lock.Lock() - r.currentStage = StageIDDeployFailed - r.state = step.RunningStepStateFinished - r.lock.Unlock() - outputID := errorStr output := any(DeployFailed{ Error: err.Error(), }) - r.stageChangeHandler.OnStepComplete( - r, - string(r.currentStage), - &outputID, - &output, - &r.wg, - ) + r.completeStage(StageIDDeployFailed, step.RunningStepStateFinished, &outputID, &output) } func (r *runningStep) startFailed(err error) { r.logger.Debugf("Start failed stage for step %s/%s", r.runID, r.pluginStepID) - r.lock.Lock() - previousStage := string(r.currentStage) - r.currentStage = StageIDCrashed - r.lock.Unlock() - - r.stageChangeHandler.OnStageChange( - r, - &previousStage, - nil, - nil, - string(r.currentStage), - false, - &r.wg, - ) + r.transitionStage(StageIDCrashed, step.RunningStepStateRunning) r.logger.Warningf("Plugin step %s/%s start failed. %v", r.runID, r.pluginStepID, err) // Now it's done. - r.lock.Lock() - r.currentStage = StageIDCrashed - r.state = step.RunningStepStateFinished - r.lock.Unlock() - outputID := errorStr output := any(Crashed{ Output: err.Error(), }) - r.stageChangeHandler.OnStepComplete( - r, - string(r.currentStage), - &outputID, - &output, - &r.wg, - ) + + r.completeStage(StageIDCrashed, step.RunningStepStateFinished, &outputID, &output) } func (r *runningStep) runFailed(err error) { r.logger.Debugf("Run failed stage for step %s/%s", r.runID, r.pluginStepID) + r.transitionStage(StageIDCrashed, step.RunningStepStateRunning) + r.logger.Warningf("Plugin step %s/%s run failed. %v", r.runID, r.pluginStepID, err) + + // Now it's done. + outputID := errorStr + output := any(Crashed{ + Output: err.Error(), + }) + r.completeStage(StageIDCrashed, step.RunningStepStateFinished, &outputID, &output) +} + +// TransitionStage transitions the stage to the specified stage, and the state to the specified state +func (r *runningStep) transitionStage(newStage StageID, state step.RunningStepState) { // A current lack of observability into the atp client prevents // non-fragile testing of this function. - r.lock.Lock() previousStage := string(r.currentStage) - r.currentStage = StageIDCrashed + r.currentStage = newStage // Don't forget to update this, or else it will behave very oddly. - // First running, then finished. You can't skip states. r.state = step.RunningStepStateRunning + // First running, then finished. You can't skip states. + r.state = state r.lock.Unlock() - r.stageChangeHandler.OnStageChange( r, &previousStage, nil, nil, - string(r.currentStage), + string(newStage), false, &r.wg, ) +} - r.logger.Warningf("Plugin step %s/%s run failed. %v", r.runID, r.pluginStepID, err) - - // Now it's done. +func (r *runningStep) completeStage(currentStage StageID, state step.RunningStepState, outputID *string, previousStageOutput *any) { r.lock.Lock() - r.currentStage = StageIDCrashed - r.state = step.RunningStepStateFinished + previousStage := string(r.currentStage) + r.currentStage = currentStage + r.state = state r.lock.Unlock() - outputID := errorStr - output := any(Crashed{ - Output: err.Error(), - }) r.stageChangeHandler.OnStepComplete( r, - string(r.currentStage), - &outputID, - &output, + previousStage, + outputID, + previousStageOutput, &r.wg, ) } From ce8c752519d343b38fd10f2b76e75e78045b4599 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Tue, 24 Oct 2023 18:20:31 -0400 Subject: [PATCH 19/54] Refactoring to fix linting errors I reduced the amount of code in one function, and reduced redundancy of unlocking --- internal/step/plugin/provider.go | 198 ++++++++++++++++--------------- workflow/executor.go | 131 ++++++++++++-------- workflow/workflow.go | 2 +- 3 files changed, 186 insertions(+), 145 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 0dee2557..a44adf30 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -169,7 +169,7 @@ var crashedLifecycleStage = step.LifecycleStage{ FinishedName: "crashed", } -// Lifecycle returns a lifecycle that contains all plugin lifecycle stages +// Lifecycle returns a lifecycle that contains all plugin lifecycle stages. func (p *pluginProvider) Lifecycle() step.Lifecycle[step.LifecycleStage] { return step.Lifecycle[step.LifecycleStage]{ InitialStage: string(StageIDDeploy), @@ -539,117 +539,126 @@ func (r *runningStep) ProvideStageInput(stage string, input map[string]any) erro // affect the counting of step states in the workflow's Execute function // and notifySteps function. r.lock.Lock() + defer r.lock.Unlock() r.logger.Debugf("ProvideStageInput START") defer r.logger.Debugf("ProvideStageInput END") - // Checks which stage it is getting input for + // Checks which stage it is getting input for, and handles it. switch stage { case string(StageIDDeploy): - // input provided on this call overwrites the deployer configuration - // set at this plugin provider's instantiation - if r.deployInputAvailable { - r.lock.Unlock() - return fmt.Errorf("deployment information provided more than once") - } - var unserializedDeployerConfig any - var err error - if input["deploy"] != nil { - unserializedDeployerConfig, err = r.deployerRegistry.Schema().Unserialize(input["deploy"]) - if err != nil { - r.lock.Unlock() - return fmt.Errorf("invalid deployment information (%w)", err) - } - } else { - r.useLocalDeployer = true - } - // Make sure we transition the state before unlocking so there are no race conditions. - - r.deployInputAvailable = true - if r.state == step.RunningStepStateWaitingForInput && r.currentStage == StageIDDeploy { - r.state = step.RunningStepStateRunning - } - - // Feed the deploy step its input. - select { - case r.deployInput <- unserializedDeployerConfig: - default: - r.lock.Unlock() - return fmt.Errorf("unable to provide input to deploy stage for step %s/%s", r.runID, r.pluginStepID) - } - r.lock.Unlock() - return nil + return r.provideDeployInput(input) case string(StageIDStarting): - if r.runInputAvailable { - r.lock.Unlock() - return fmt.Errorf("input provided more than once") - } - // Ensure input is given - if input["input"] == nil { - r.lock.Unlock() - return fmt.Errorf("bug: invalid input for 'running' stage, expected 'input' field") - } - // Validate the input by unserializing it - if _, err := r.stepSchema.Input().Unserialize(input["input"]); err != nil { - r.lock.Unlock() - return err - } - // Make sure we transition the state before unlocking so there are no race conditions. - r.runInputAvailable = true - - // Unlock before passing the data over the channel to prevent a deadlock. - // The other end of the channel needs to be unlocked to read the data. - - // Feed the run step its input over the channel. - select { - case r.runInput <- input["input"]: - default: - r.lock.Unlock() - return fmt.Errorf("unable to provide input to run stage for step %s/%s", r.runID, r.pluginStepID) - } - r.lock.Unlock() - return nil + return r.provideStartingInput(input) case string(StageIDRunning): - r.lock.Unlock() return nil case string(StageIDCancelled): - if input["stop_if"] != false && input["stop_if"] != nil { - r.logger.Infof("Cancelling step %s/%s", r.runID, r.pluginStepID) - - // Verify that the step has a cancel signal - cancelSignal := r.stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] - if cancelSignal == nil { - r.logger.Errorf("could not cancel step %s/%s. Does not contain cancel signal receiver.", r.runID, r.pluginStepID) - } else if err := plugin.CancellationSignalSchema.DataSchema().ValidateCompatibility(cancelSignal.DataSchema()); err != nil { - r.logger.Errorf("validation failed for cancel signal for step %s/%s: %s", r.runID, r.pluginStepID, err) - } else { - // Canceling the context should be enough when the stage isn't running. - if r.currentStage == StageIDRunning { - // Validated. Now call the signal. - r.signalToStep <- schema.Input{RunID: r.runID, ID: cancelSignal.ID(), InputData: map[any]any{}} - } - } - // Now cancel the context to stop - r.cancel() - } - r.lock.Unlock() - return nil + return r.provideCancelledInput(input) case string(StageIDDeployFailed): - r.lock.Unlock() return nil case string(StageIDCrashed): - r.lock.Unlock() return nil case string(StageIDOutput): - r.lock.Unlock() return nil default: - r.lock.Unlock() return fmt.Errorf("bug: invalid stage: %s", stage) } } -// ForceClose closes the step without waiting for a graceful shutdown of the ATP client -// Warning: This means that +func (r *runningStep) provideDeployInput(input map[string]any) error { + // Note: The calling function must have the step mutex locked + // input provided on this call overwrites the deployer configuration + // set at this plugin provider's instantiation + if r.deployInputAvailable { + return fmt.Errorf("deployment information provided more than once") + } + var unserializedDeployerConfig any + var err error + if input["deploy"] != nil { + unserializedDeployerConfig, err = r.deployerRegistry.Schema().Unserialize(input["deploy"]) + if err != nil { + return fmt.Errorf("invalid deployment information (%w)", err) + } + } else { + r.useLocalDeployer = true + } + // Make sure we transition the state before unlocking so there are no race conditions. + + r.deployInputAvailable = true + if r.state == step.RunningStepStateWaitingForInput && r.currentStage == StageIDDeploy { + r.state = step.RunningStepStateRunning + } + + // Feed the deploy step its input. + select { + case r.deployInput <- unserializedDeployerConfig: + default: + return fmt.Errorf("unable to provide input to deploy stage for step %s/%s", r.runID, r.pluginStepID) + } + return nil +} + +func (r *runningStep) provideStartingInput(input map[string]any) error { + // Note: The calling function must have the step mutex locked + if r.runInputAvailable { + return fmt.Errorf("input provided more than once") + } + // Ensure input is given + if input["input"] == nil { + return fmt.Errorf("bug: invalid input for 'running' stage, expected 'input' field") + } + // Validate the input by unserializing it + if _, err := r.stepSchema.Input().Unserialize(input["input"]); err != nil { + return err + } + // Make sure we transition the state before unlocking so there are no race conditions. + r.runInputAvailable = true + + // Unlock before passing the data over the channel to prevent a deadlock. + // The other end of the channel needs to be unlocked to read the data. + + // Feed the run step its input over the channel. + select { + case r.runInput <- input["input"]: + default: + return fmt.Errorf("unable to provide input to run stage for step %s/%s", r.runID, r.pluginStepID) + } + return nil +} + +func (r *runningStep) provideCancelledInput(input map[string]any) error { + // Note: The calling function must have the step mutex locked + // Cancel if the step field is present and isn't false + if input["stop_if"] != false && input["stop_if"] != nil { + r.cancelStep() + } + return nil +} + +// cancelStep gracefully requests cancellation for any stage. +// If running, it sends a cancel signal if the plugin supports it. +func (r *runningStep) cancelStep() { + r.logger.Infof("Cancelling step %s/%s", r.runID, r.pluginStepID) + // We only need to call the signal if the step is running. + // If it isn't, cancelling the context alone should be enough. + if r.currentStage == StageIDRunning { + // Verify that the step has a cancel signal + cancelSignal := r.stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] + if cancelSignal == nil { + r.logger.Errorf("could not cancel step %s/%s. Does not contain cancel signal receiver.", r.runID, r.pluginStepID) + } else if err := plugin.CancellationSignalSchema.DataSchema().ValidateCompatibility(cancelSignal.DataSchema()); err != nil { + r.logger.Errorf("validation failed for cancel signal for step %s/%s: %s", r.runID, r.pluginStepID, err) + } else { + // Validated. Now call the signal. + r.signalToStep <- schema.Input{RunID: r.runID, ID: cancelSignal.ID(), InputData: map[any]any{}} + } + } + // Now cancel the context to stop the non-running parts of the step + r.cancel() +} + +// ForceClose closes the step without waiting for a graceful shutdown of the ATP client. +// Warning: This means that it won't wait for the ATP client to finish. This is okay if using a deployer that +// will stop execution once the deployer closes it. func (r *runningStep) ForceClose() error { err := r.closeComponents(false) // Wait for the run to finish to ensure that it's not running after closing. @@ -943,7 +952,9 @@ func (r *runningStep) runFailed(err error) { r.completeStage(StageIDCrashed, step.RunningStepStateFinished, &outputID, &output) } -// TransitionStage transitions the stage to the specified stage, and the state to the specified state +// TransitionStage transitions the stage to the specified stage, and the state to the specified state. +// +//nolint:unparam func (r *runningStep) transitionStage(newStage StageID, state step.RunningStepState) { // A current lack of observability into the atp client prevents // non-fragile testing of this function. @@ -965,6 +976,7 @@ func (r *runningStep) transitionStage(newStage StageID, state step.RunningStepSt ) } +//nolint:unparam func (r *runningStep) completeStage(currentStage StageID, state step.RunningStepState, outputID *string, previousStageOutput *any) { r.lock.Lock() previousStage := string(r.currentStage) diff --git a/workflow/executor.go b/workflow/executor.go index af97f7bd..f1a4add4 100644 --- a/workflow/executor.go +++ b/workflow/executor.go @@ -127,7 +127,7 @@ func (e *executor) Prepare(workflow *Workflow, workflowContext map[string][]byte // through them again to verify that all inputs are valid. So verify that all required inputs are present, schemas // are valid, etc. // Do this by looping through the steps' inputs, then verifying that the dag can provide them. - if err := e.verifyStageInputs(workflow, workflowContext, stepLifecycles, dag, internalDataModel); err != nil { + if err := e.verifyWorkflowStageInputs(workflow, workflowContext, stepLifecycles, dag, internalDataModel); err != nil { return nil, err } // Stage 6: The output data model @@ -332,74 +332,103 @@ func (e *executor) connectStepDependencies( return nil } -// verifyStageInputs verifies the schemas of the step inputs -func (e *executor) verifyStageInputs( +// verifyWorkflowStageInputs verifies the schemas of the step inputs. +func (e *executor) verifyWorkflowStageInputs( workflow *Workflow, workflowContext map[string][]byte, stepLifecycles map[string]step.Lifecycle[step.LifecycleStageWithSchema], dag dgraph.DirectedGraph[*DAGItem], internalDataModel *schema.ScopeSchema, ) error { - for stepID, _ /*stepData*/ := range workflow.Steps { + // Loop through all the steps in the engine + for stepID /*stepData is the unused key*/ := range workflow.Steps { + // Then loop through the stages of that step lifecycle := stepLifecycles[stepID] for _, stage := range lifecycle.Stages { - currentStageNode, err := dag.GetNodeByID(GetStageNodeID(stepID, stage.ID)) + err := e.verifyStageInputs(dag, stepID, stage, workflowContext, internalDataModel) if err != nil { - return fmt.Errorf("bug: node for current stage not found (%w)", err) + return err } - // stageData provides the info needed for this node, without the expressions resolved. - stageData := currentStageNode.Item().Data - - // Use reflection to convert the stage's input data to a readable map. - parsedInputs := make(map[string]any) - if stageData != nil { - v := reflect.ValueOf(stageData) - if v.Kind() != reflect.Map { - return fmt.Errorf("could not validate input. Stage data is not a map. It is %s", v.Kind()) - } + } + } + return nil +} - for _, reflectedKey := range v.MapKeys() { - if reflectedKey.Kind() != reflect.Interface { - return fmt.Errorf("expected input key to be interface of a string. Got %s", reflectedKey.Kind()) - } - // Now convert interface to string - key, ok := reflectedKey.Interface().(string) - if !ok { - return fmt.Errorf("error converting input key to string") - } - value := v.MapIndex(reflectedKey).Interface() - parsedInputs[key] = value - } +func (e *executor) verifyStageInputs( + dag dgraph.DirectedGraph[*DAGItem], + stepID string, + stage step.LifecycleStageWithSchema, + workflowContext map[string][]byte, + internalDataModel *schema.ScopeSchema, +) error { + // First, get the parsed inputs of the stage + parsedInputs, err := e.getStageInputs(dag, stepID, stage) + if err != nil { + return err + } + // Next, loop through the input schema fields. + for name, stageInputSchema := range stage.InputSchema { + providedInputForField := parsedInputs[name] + // Check if the field is present in the stage data. + // If it is NOT present and is NOT required, continue to next field. + // If it is NOT present and IS required, fail + // If it IS present, verify whether schema is compatible with the schema of the provided data, + // then notify the provider that the data is present. + // This is running pre-workflow run, so you can check the schemas, but most fields won't be able to be + // resolved to an actual value. + if providedInputForField == nil { + // not present + if stageInputSchema.RequiredValue { + return fmt.Errorf("required input %s of type %s not found for step %s", + name, stageInputSchema.TypeID(), stepID) } - // Next, loop through the input schema fields. - for name, stageInputSchema := range stage.InputSchema { - providedInputForField := parsedInputs[name] - // Check if the field is present in the stage data. - // If it is NOT present and is NOT required, continue to next field. - // If it is NOT present and IS required, fail - // If it IS present, verify whether schema is compatible with the schema of the provided data, - // then notify the provider that the data is present. - // This is running pre-workflow run, so you can check the schemas, but most fields won't be able to be - // resolved to an actual value. - if providedInputForField == nil { - // not present - if stageInputSchema.RequiredValue { - return fmt.Errorf("required input %s of type %s not found for step %s", - name, stageInputSchema.TypeID(), stepID) - } - } else { - // It is present, so make sure it is compatible. - err := e.preValidateCompatibility(internalDataModel, providedInputForField, stageInputSchema, workflowContext) - if err != nil { - return fmt.Errorf("input validation failed for workflow step %s stage %s (%w)", stepID, stage.ID, err) - } - } + } else { + // It is present, so make sure it is compatible. + err := e.preValidateCompatibility(internalDataModel, providedInputForField, stageInputSchema, workflowContext) + if err != nil { + return fmt.Errorf("input validation failed for workflow step %s stage %s (%w)", stepID, stage.ID, err) } } } return nil } +func (e *executor) getStageInputs( + dag dgraph.DirectedGraph[*DAGItem], + stepID string, + stage step.LifecycleStageWithSchema, +) (map[string]any, error) { + currentStageNode, err := dag.GetNodeByID(GetStageNodeID(stepID, stage.ID)) + if err != nil { + return nil, fmt.Errorf("bug: node for current stage not found (%w)", err) + } + // stageData provides the info needed for this node, without the expressions resolved. + stageData := currentStageNode.Item().Data + + // Use reflection to convert the stage's input data to a readable map. + parsedInputs := make(map[string]any) + if stageData != nil { + v := reflect.ValueOf(stageData) + if v.Kind() != reflect.Map { + return nil, fmt.Errorf("could not validate input. Stage data is not a map. It is %s", v.Kind()) + } + + for _, reflectedKey := range v.MapKeys() { + if reflectedKey.Kind() != reflect.Interface { + return nil, fmt.Errorf("expected input key to be interface of a string. Got %s", reflectedKey.Kind()) + } + // Now convert interface to string + key, ok := reflectedKey.Interface().(string) + if !ok { + return nil, fmt.Errorf("error converting input key to string") + } + value := v.MapIndex(reflectedKey).Interface() + parsedInputs[key] = value + } + } + return parsedInputs, nil +} + func (e *executor) preValidateCompatibility(rootSchema schema.Scope, inputField any, propertySchema *schema.PropertySchema, workflowContext map[string][]byte) error { // Get the type/value structure diff --git a/workflow/workflow.go b/workflow/workflow.go index a51510ee..b311cc87 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -533,7 +533,7 @@ func (l *loopState) resolveExpressions(inputData any, dataModel any) (any, error } } -// stageChangeHandler is implementing step.StageChangeHandler +// stageChangeHandler is implementing step.StageChangeHandler. type stageChangeHandler struct { onStageChange func( step step.RunningStep, From ec46f7b1882b72a687d0dba56b578e5845b149ea Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Tue, 24 Oct 2023 18:31:46 -0400 Subject: [PATCH 20/54] Fix linting errors These ones were only shown on CI, not locally --- internal/step/plugin/provider.go | 6 ++++-- workflow/workflow.go | 17 +++++++++++++++-- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index a44adf30..71fc0354 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -692,8 +692,10 @@ func (r *runningStep) closeComponents(closeATP bool) error { } r.container = nil r.lock.Unlock() - if containerErr != nil || atpErr != nil { - return fmt.Errorf("failed to stop atp client (%w) or container (%w)", atpErr, containerErr) + if containerErr != nil { + return fmt.Errorf("error while stopping container (%w)", containerErr) + } else if atpErr != nil { + return fmt.Errorf("error while stopping atp client (%w)", atpErr) // Do not wait in this case. It may never get resolved. } return nil diff --git a/workflow/workflow.go b/workflow/workflow.go index b311cc87..9ffab54d 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -408,8 +408,15 @@ func (l *loopState) notifySteps() { //nolint:gocognit } } -func (l *loopState) checkForDeadlocks(retries int, wg *sync.WaitGroup) { - // Here we make sure we don't have a deadlock. +type stateCounters struct { + starting int + waitingWithInbound int + waitingWithoutInbound int + running int + finished int +} + +func (l *loopState) countStates() stateCounters { counters := struct { starting int waitingWithInbound int @@ -467,6 +474,12 @@ func (l *loopState) checkForDeadlocks(retries int, wg *sync.WaitGroup) { l.logger.Debugf("Step %s is currently finished.", stepID) } } + return counters +} + +func (l *loopState) checkForDeadlocks(retries int, wg *sync.WaitGroup) { + // Here we make sure we don't have a deadlock. + counters := l.countStates() l.logger.Infof( "There are currently %d steps starting, %d waiting for input, %d ready for input, %d running, %d finished", counters.starting, From 58e5bf0b203522170e29c853af886e7a8acfd5c3 Mon Sep 17 00:00:00 2001 From: Dustin Black Date: Wed, 25 Oct 2023 12:01:15 +0200 Subject: [PATCH 21/54] bump internal dependencies --- go.mod | 11 ++++++----- go.sum | 30 ++++++++++++++++-------------- 2 files changed, 22 insertions(+), 19 deletions(-) diff --git a/go.mod b/go.mod index 20fd4c59..327dd35f 100644 --- a/go.mod +++ b/go.mod @@ -9,12 +9,12 @@ require ( go.arcalot.io/log/v2 v2.0.0 go.flow.arcalot.io/deployer v0.2.0 go.flow.arcalot.io/dockerdeployer v0.3.0 - go.flow.arcalot.io/expressions v0.2.0 + go.flow.arcalot.io/expressions v0.2.1 go.flow.arcalot.io/kubernetesdeployer v0.5.1 - go.flow.arcalot.io/pluginsdk v0.5.0-beta.2 - go.flow.arcalot.io/podmandeployer v0.3.1 + go.flow.arcalot.io/pluginsdk v0.5.0 + go.flow.arcalot.io/podmandeployer v0.4.1 go.flow.arcalot.io/pythondeployer v0.2.0 - go.flow.arcalot.io/testdeployer v0.3.0-beta.1 + go.flow.arcalot.io/testdeployer v0.3.0 gopkg.in/yaml.v3 v3.0.1 ) @@ -23,7 +23,7 @@ require ( github.com/Microsoft/go-winio v0.6.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/docker/distribution v2.8.2+incompatible // indirect - github.com/docker/docker v24.0.5+incompatible // indirect + github.com/docker/docker v24.0.6+incompatible // indirect github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/emicklei/go-restful/v3 v3.10.0 // indirect @@ -47,6 +47,7 @@ require ( github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.0.2 // indirect github.com/pkg/errors v0.9.1 // indirect + github.com/rogpeppe/go-internal v1.11.0 // indirect github.com/x448/float16 v0.8.4 // indirect go.flow.arcalot.io/testplugin v0.2.1 // indirect golang.org/x/mod v0.13.0 // indirect diff --git a/go.sum b/go.sum index f64f0bce..4b519aee 100644 --- a/go.sum +++ b/go.sum @@ -21,8 +21,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/docker/distribution v2.8.2+incompatible h1:T3de5rq0dB1j30rp0sA2rER+m322EBzniBPB6ZIzuh8= github.com/docker/distribution v2.8.2+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= -github.com/docker/docker v24.0.5+incompatible h1:WmgcE4fxyI6EEXxBRxsHnZXrO1pQ3smi0k/jho4HLeY= -github.com/docker/docker v24.0.5+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v24.0.6+incompatible h1:hceabKCtUgDqPu+qm0NgsaXf28Ljf4/pWFL7xjWWDgE= +github.com/docker/docker v24.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= @@ -90,7 +90,7 @@ github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/imdario/mergo v0.3.6 h1:xTNEAn+kxVO7dTZGu0CegyqKZmoWFI0rF8UxjlB2d28= -github.com/joho/godotenv v1.4.0 h1:3l4+N6zfMWnkbPEXKng2o2/MR5mSwTrBih4ZEkkz1lg= +github.com/joho/godotenv v1.5.1 h1:7eLL/+HRGLY0ldzfGMeQkb7vMd0as4CfYvUVzLqw0N0= github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= @@ -98,8 +98,8 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -129,6 +129,8 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= +github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= +github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= @@ -139,8 +141,8 @@ github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UV github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= @@ -160,18 +162,18 @@ go.flow.arcalot.io/deployer v0.2.0 h1:CpkCYlB8NfpmELIEPdw3/al8XknCSfD/L2vie2lJBJ go.flow.arcalot.io/deployer v0.2.0/go.mod h1:xVSB+svHVPmX6yTZIU0K4U/pDbs+rezsWa69vYA+E6k= go.flow.arcalot.io/dockerdeployer v0.3.0 h1:9F9ZyaiB1kwKLRMXAktkyD3eDVvWoMYSHCojTrPDusk= go.flow.arcalot.io/dockerdeployer v0.3.0/go.mod h1:9cMMHmkHxzHgLOLva6RmGb5d8jrbRkar/PcjeiU0QvA= -go.flow.arcalot.io/expressions v0.2.0 h1:2k8InnpLqVmv5SDvJ1xRz1ubqF+zKN44U08D50TkqJs= -go.flow.arcalot.io/expressions v0.2.0/go.mod h1:m4p6oCwjjxRjPCAYGeZDNP9DGfoDODHo1z4pBoO+Lpc= +go.flow.arcalot.io/expressions v0.2.1 h1:TAAbDrgJJLpmgA5ASyP/KzrXWtpEaQ8JsCPHgpe5kLw= +go.flow.arcalot.io/expressions v0.2.1/go.mod h1:Vw1ScNu4Uyw1/l87LAH8jxe0DyRWwMh+rlfB/BPYDOU= go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEXBAnyoWJnHls= go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= -go.flow.arcalot.io/pluginsdk v0.5.0-beta.2 h1:y69ZKZKghAV4cb3w2XPGFculRo+3mWxwJXX2WPfw0qQ= -go.flow.arcalot.io/pluginsdk v0.5.0-beta.2/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= -go.flow.arcalot.io/podmandeployer v0.3.1 h1:AbRmTTtuK50PLkZyu194oSra0zUCeM3lDCWTc7oo4Ys= -go.flow.arcalot.io/podmandeployer v0.3.1/go.mod h1:SmROc9nHG+KfKasyTeKtGmI9EBlXCupXjBIgX5glGn8= +go.flow.arcalot.io/pluginsdk v0.5.0 h1:TRS/waCTcdoMZ9neDAcfy3zpzyDnPHRbhV+Y1kpcw3Y= +go.flow.arcalot.io/pluginsdk v0.5.0/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= +go.flow.arcalot.io/podmandeployer v0.4.1 h1:TnGmYSilOcSZj8dn98kvsnEC4q+6Npu/ejepIUJXlQw= +go.flow.arcalot.io/podmandeployer v0.4.1/go.mod h1:dWLUNpbdnZAqy1S4oWb/0cLNI1PI2Z3Eupn0o/yWNUw= go.flow.arcalot.io/pythondeployer v0.2.0 h1:iSVxQzRGrEr0/bJfsRNQ7Q3ItA3sr0+7jDg1SL15llo= go.flow.arcalot.io/pythondeployer v0.2.0/go.mod h1:zegeDjxiddprrPmO6243AslQX7BxCmV46xXSUG5TZGc= -go.flow.arcalot.io/testdeployer v0.3.0-beta.1 h1:OgTdbOnqzmLdwG/GVnnYBxsrgqD1hySDjSm54j6Gvlo= -go.flow.arcalot.io/testdeployer v0.3.0-beta.1/go.mod h1:qTd2I5m46jR78e5ivoQ94sTNNFsJRLROScQ35+v9TNE= +go.flow.arcalot.io/testdeployer v0.3.0 h1:Soyz+rDa3Y3VjWBGuL3zNlX3LM4uKp9Ex7///fCgrZA= +go.flow.arcalot.io/testdeployer v0.3.0/go.mod h1:Eel0ORhtKdYYDsd+e+btBBygIn+9Sz/b+JFDwH39VWI= go.flow.arcalot.io/testplugin v0.2.1 h1:9kQ2MKvcXtEcwk5c4qSWN+FovpER2C9vn730laAm9iE= go.flow.arcalot.io/testplugin v0.2.1/go.mod h1:ZoVF8tIKppQmj5nvoZPA48GQ7BuoWXQcuCw2x2sJxjE= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= @@ -295,7 +297,7 @@ gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gotest.tools/v3 v3.4.0 h1:ZazjZUfuVeZGLAmlKKuyv3IKP5orXcwtOwDQH6YVr6o= +gotest.tools/v3 v3.5.1 h1:EENdUnS3pdur5nybKYIh2Vfgc8IUNBjxDPSjtiJcOzU= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= k8s.io/api v0.25.4 h1:3YO8J4RtmG7elEgaWMb4HgmpS2CfY1QlaOz9nwB+ZSs= From 3fe61d5603e778b787c2839a0b6c8431d1ea288a Mon Sep 17 00:00:00 2001 From: Dustin Black Date: Wed, 25 Oct 2023 14:49:14 +0200 Subject: [PATCH 22/54] update all internal dependencies --- go.mod | 13 +++++++------ go.sum | 28 +++++++++++++++------------- 2 files changed, 22 insertions(+), 19 deletions(-) diff --git a/go.mod b/go.mod index 327dd35f..5ac86981 100644 --- a/go.mod +++ b/go.mod @@ -7,12 +7,12 @@ require ( go.arcalot.io/dgraph v1.1.0 go.arcalot.io/lang v1.0.0 go.arcalot.io/log/v2 v2.0.0 - go.flow.arcalot.io/deployer v0.2.0 - go.flow.arcalot.io/dockerdeployer v0.3.0 + go.flow.arcalot.io/deployer v0.3.0 + go.flow.arcalot.io/dockerdeployer v0.4.0 go.flow.arcalot.io/expressions v0.2.1 - go.flow.arcalot.io/kubernetesdeployer v0.5.1 + go.flow.arcalot.io/kubernetesdeployer v0.7.0 go.flow.arcalot.io/pluginsdk v0.5.0 - go.flow.arcalot.io/podmandeployer v0.4.1 + go.flow.arcalot.io/podmandeployer v0.5.0 go.flow.arcalot.io/pythondeployer v0.2.0 go.flow.arcalot.io/testdeployer v0.3.0 gopkg.in/yaml.v3 v3.0.1 @@ -22,7 +22,8 @@ require ( github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/docker/distribution v2.8.2+incompatible // indirect + github.com/distribution/reference v0.5.0 // indirect + github.com/docker/distribution v2.8.3+incompatible // indirect github.com/docker/docker v24.0.6+incompatible // indirect github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-units v0.5.0 // indirect @@ -56,7 +57,7 @@ require ( golang.org/x/sys v0.12.0 // indirect golang.org/x/term v0.12.0 // indirect golang.org/x/text v0.13.0 // indirect - golang.org/x/time v0.2.0 // indirect + golang.org/x/time v0.3.0 // indirect golang.org/x/tools v0.13.0 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/protobuf v1.28.1 // indirect diff --git a/go.sum b/go.sum index 4b519aee..90eef39b 100644 --- a/go.sum +++ b/go.sum @@ -19,8 +19,10 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ3 github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/docker/distribution v2.8.2+incompatible h1:T3de5rq0dB1j30rp0sA2rER+m322EBzniBPB6ZIzuh8= -github.com/docker/distribution v2.8.2+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= +github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= +github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= +github.com/docker/distribution v2.8.3+incompatible h1:AtKxIZ36LoNK51+Z6RpzLpddBirtxJnzDrHLEKxTAYk= +github.com/docker/distribution v2.8.3+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= github.com/docker/docker v24.0.6+incompatible h1:hceabKCtUgDqPu+qm0NgsaXf28Ljf4/pWFL7xjWWDgE= github.com/docker/docker v24.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= @@ -108,7 +110,7 @@ github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0 github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= github.com/moby/spdystream v0.2.0 h1:cjW1zVyyoiM0T7b6UoySUFqzXMoqRckQtXwGPiBhOM8= github.com/moby/spdystream v0.2.0/go.mod h1:f7i0iNDQJ059oMTcWxx8MA/zKFIuD/lY+0GqbN2Wy8c= -github.com/moby/term v0.0.0-20221105221325-4eb28fa6025c h1:RC8WMpjonrBfyAh6VN/POIPtYD5tRAq0qMqCRjQNK+g= +github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -158,18 +160,18 @@ go.arcalot.io/lang v1.0.0 h1:mgDaieT4wWdZTnR4V7+/pgYRmzfU7VZZgIzHccuxAbY= go.arcalot.io/lang v1.0.0/go.mod h1:ALqfYEhAzC2WoGLaycmJoNJd5NmkR7V1PSKp/c5D278= go.arcalot.io/log/v2 v2.0.0 h1:mbmsWDVBXZNWrDzUh5JLzeGCQ59kTuMFs+pyfJGc1hk= go.arcalot.io/log/v2 v2.0.0/go.mod h1:1V8jnFIIGwh2CtcGkHNOmy1nCo7LbazQNkUcnKYNMn4= -go.flow.arcalot.io/deployer v0.2.0 h1:CpkCYlB8NfpmELIEPdw3/al8XknCSfD/L2vie2lJBJo= -go.flow.arcalot.io/deployer v0.2.0/go.mod h1:xVSB+svHVPmX6yTZIU0K4U/pDbs+rezsWa69vYA+E6k= -go.flow.arcalot.io/dockerdeployer v0.3.0 h1:9F9ZyaiB1kwKLRMXAktkyD3eDVvWoMYSHCojTrPDusk= -go.flow.arcalot.io/dockerdeployer v0.3.0/go.mod h1:9cMMHmkHxzHgLOLva6RmGb5d8jrbRkar/PcjeiU0QvA= +go.flow.arcalot.io/deployer v0.3.0 h1:LPikgRG5jGA76W8JthycvzfREL5Y0+++KAiQxSnKhdU= +go.flow.arcalot.io/deployer v0.3.0/go.mod h1:x6gsz/hANR8qN1nerpyY3vXpdaqofDH5Wlg+Nsqg/x0= +go.flow.arcalot.io/dockerdeployer v0.4.0 h1:t5b8o3xfKKb/WIX558486csjo4uMQmAXsikBLsKFEIg= +go.flow.arcalot.io/dockerdeployer v0.4.0/go.mod h1:UZSM6buJBRlgCURUE/BVkak8tfAXzj3oeQBSRZECbSc= go.flow.arcalot.io/expressions v0.2.1 h1:TAAbDrgJJLpmgA5ASyP/KzrXWtpEaQ8JsCPHgpe5kLw= go.flow.arcalot.io/expressions v0.2.1/go.mod h1:Vw1ScNu4Uyw1/l87LAH8jxe0DyRWwMh+rlfB/BPYDOU= -go.flow.arcalot.io/kubernetesdeployer v0.5.1 h1:YpPFSouEWjxN8sxdr4BuZMl3IM9jCGEXBAnyoWJnHls= -go.flow.arcalot.io/kubernetesdeployer v0.5.1/go.mod h1:mdhwBGQ0wlquo+wUR0VNSAyj9NYxDdTwM7vYV1XmRnw= +go.flow.arcalot.io/kubernetesdeployer v0.7.0 h1:r41qWc/XiPy9l3cfMXZG8F2kGenRh1xsx2auim/Ydyw= +go.flow.arcalot.io/kubernetesdeployer v0.7.0/go.mod h1:VvU6duoo5NR2ITUhx/UCGrkdJnXIeYm+/yHmGKtkXsk= go.flow.arcalot.io/pluginsdk v0.5.0 h1:TRS/waCTcdoMZ9neDAcfy3zpzyDnPHRbhV+Y1kpcw3Y= go.flow.arcalot.io/pluginsdk v0.5.0/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= -go.flow.arcalot.io/podmandeployer v0.4.1 h1:TnGmYSilOcSZj8dn98kvsnEC4q+6Npu/ejepIUJXlQw= -go.flow.arcalot.io/podmandeployer v0.4.1/go.mod h1:dWLUNpbdnZAqy1S4oWb/0cLNI1PI2Z3Eupn0o/yWNUw= +go.flow.arcalot.io/podmandeployer v0.5.0 h1:h7hEhWUgxJzNKlEohZ+meKhl3FWjaXQahQ8vN3YVRNs= +go.flow.arcalot.io/podmandeployer v0.5.0/go.mod h1:36JCcTB6nauahcXUPfIpdEw7Zfp0ufM07o3VNTvrCc0= go.flow.arcalot.io/pythondeployer v0.2.0 h1:iSVxQzRGrEr0/bJfsRNQ7Q3ItA3sr0+7jDg1SL15llo= go.flow.arcalot.io/pythondeployer v0.2.0/go.mod h1:zegeDjxiddprrPmO6243AslQX7BxCmV46xXSUG5TZGc= go.flow.arcalot.io/testdeployer v0.3.0 h1:Soyz+rDa3Y3VjWBGuL3zNlX3LM4uKp9Ex7///fCgrZA= @@ -236,8 +238,8 @@ golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -golang.org/x/time v0.2.0 h1:52I/1L54xyEQAYdtcSuxtiT84KGYTBGXwayxmIpNJhE= -golang.org/x/time v0.2.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= From 15c4fce68128e5a55e7c112115b734074ed02db1 Mon Sep 17 00:00:00 2001 From: mleader Date: Tue, 17 Oct 2023 16:47:54 -0400 Subject: [PATCH 23/54] refactor config to use multiple local deployers --- config/config.go | 9 ++++++-- config/load_test.go | 8 +++---- config/schema.go | 51 ++++++++++++++++++++++++++++++++++++++++----- steps.go | 2 +- 4 files changed, 58 insertions(+), 12 deletions(-) diff --git a/config/config.go b/config/config.go index 55e45cbc..657f7410 100644 --- a/config/config.go +++ b/config/config.go @@ -15,11 +15,16 @@ type Config struct { // TypeHintPlugins holds a list of plugins that will be used when building a type hint (e.g. JSONSchema) file for // workflows. TypeHintPlugins []string `json:"plugins" yaml:"plugins"` - // LocalDeployer holds the configuration for executing plugins locally. This deployer is used to obtain the schema + // LocalDeployers holds the configuration for executing plugins locally. This deployer is used to obtain the schema // from the plugins before executing them in a remote environment. - LocalDeployer any `json:"deployer" yaml:"deployer"` + LocalDeployers LocalDeployers `json:"deployers" yaml:"deployers"` // Log configures logging for workflow runs. Log log.Config `json:"log" yaml:"log"` // StepOutputLogging allows logging of step output LoggedOutputConfigs map[string]*StepOutputLogConfig `json:"logged_outputs" yaml:"logged_outputs"` } + +type LocalDeployers struct { + image string `json:"image" yaml:"image"` + python string `json:"python" yaml:"python"` +} diff --git a/config/load_test.go b/config/load_test.go index 674cd1fb..6015de80 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -18,7 +18,7 @@ var configLoadData = map[string]struct { input: "", expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployer: map[string]any{ + LocalDeployers: map[string]any{ "type": "docker", }, Log: log.Config{ @@ -34,7 +34,7 @@ log: `, expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployer: map[string]any{ + LocalDeployers: map[string]any{ "type": "docker", }, Log: log.Config{ @@ -50,7 +50,7 @@ deployer: `, expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployer: map[string]any{ + LocalDeployers: map[string]any{ "type": "kubernetes", }, Log: log.Config{ @@ -68,7 +68,7 @@ plugins: TypeHintPlugins: []string{ "quay.io/arcalot/example-plugin:latest", }, - LocalDeployer: map[string]any{ + LocalDeployers: map[string]any{ "type": "docker", }, Log: log.Config{ diff --git a/config/schema.go b/config/schema.go index ae940ea1..afdfc24c 100644 --- a/config/schema.go +++ b/config/schema.go @@ -45,12 +45,12 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, ), - "deployer": schema.NewPropertySchema( - schema.NewAnySchema(), + "deployers": schema.NewPropertySchema( + schema.NewRefSchema("LocalDeployers", nil), schema.NewDisplayValue( - schema.PointerTo("Local deployer"), + schema.PointerTo("Local deployers"), schema.PointerTo( - "Local container environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", + "Default deployers for each plugin type.", ), nil, ), @@ -58,7 +58,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, nil, - schema.PointerTo("{\"type\":\"docker\"}"), + nil, nil, ), "logged_outputs": schema.NewPropertySchema( @@ -157,5 +157,46 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { ), }, ), + schema.NewStructMappedObjectSchema[*LocalDeployers]( + "LocalDeployers", + map[string]*schema.PropertySchema{ + "image": schema.NewPropertySchema( + schema.NewAnySchema(), + schema.NewDisplayValue( + schema.PointerTo("Local image deployer"), + schema.PointerTo( + "Local container environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", + ), + nil, + ), + //true, + false, + nil, + nil, + nil, + //schema.PointerTo("{\"type\":\"docker\"}"), + nil, + nil, + ), + "python": schema.NewPropertySchema( + schema.NewAnySchema(), + schema.NewDisplayValue( + schema.PointerTo("Local python deployer"), + schema.PointerTo( + "Local python environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", + ), + nil, + ), + //true, + false, + nil, + nil, + nil, + //schema.PointerTo("{\"type\":\"python\"}"), + nil, + nil, + ), + }, + ), ) } diff --git a/steps.go b/steps.go index db1f382a..6ed80f8f 100644 --- a/steps.go +++ b/steps.go @@ -19,7 +19,7 @@ func NewDefaultStepRegistry( deployerRegistry deployerRegistry.Registry, config *config.Config, ) (step.Registry, error) { - pluginProvider, err := plugin.New(logger, deployerRegistry, config.LocalDeployer) + pluginProvider, err := plugin.New(logger, deployerRegistry, config.LocalDeployers) if err != nil { return nil, fmt.Errorf("failed to create plugin step provider (%w)", err) } From dbc3fb81aa274b00deb8bbaa14a33f6919d04bfa Mon Sep 17 00:00:00 2001 From: mleader Date: Wed, 25 Oct 2023 15:32:11 -0400 Subject: [PATCH 24/54] config defaults --- config/config.go | 4 ++-- config/load_test.go | 20 ++++++++++---------- config/schema.go | 17 ++++++++--------- 3 files changed, 20 insertions(+), 21 deletions(-) diff --git a/config/config.go b/config/config.go index 657f7410..0389d69e 100644 --- a/config/config.go +++ b/config/config.go @@ -25,6 +25,6 @@ type Config struct { } type LocalDeployers struct { - image string `json:"image" yaml:"image"` - python string `json:"python" yaml:"python"` + Image any `json:"image" yaml:"image"` + Python any `json:"python" yaml:"python"` } diff --git a/config/load_test.go b/config/load_test.go index 6015de80..8cf54537 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -18,8 +18,8 @@ var configLoadData = map[string]struct { input: "", expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployers: map[string]any{ - "type": "docker", + LocalDeployers: config.LocalDeployers{ + Image: "docker", }, Log: log.Config{ Level: log.LevelInfo, @@ -34,8 +34,8 @@ log: `, expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployers: map[string]any{ - "type": "docker", + LocalDeployers: config.LocalDeployers{ + Image: "docker", }, Log: log.Config{ Level: log.LevelDebug, @@ -45,13 +45,13 @@ log: }, "type-kubernetes": { input: ` -deployer: - type: kubernetes +deployers: + image: kubernetes `, expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployers: map[string]any{ - "type": "kubernetes", + LocalDeployers: config.LocalDeployers{ + Image: "kubernetes", }, Log: log.Config{ Level: log.LevelInfo, @@ -68,8 +68,8 @@ plugins: TypeHintPlugins: []string{ "quay.io/arcalot/example-plugin:latest", }, - LocalDeployers: map[string]any{ - "type": "docker", + LocalDeployers: config.LocalDeployers{ + Image: "docker", }, Log: log.Config{ Level: log.LevelInfo, diff --git a/config/schema.go b/config/schema.go index afdfc24c..086908b2 100644 --- a/config/schema.go +++ b/config/schema.go @@ -54,7 +54,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { ), nil, ), - true, + false, nil, nil, nil, @@ -157,13 +157,13 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { ), }, ), - schema.NewStructMappedObjectSchema[*LocalDeployers]( + schema.NewStructMappedObjectSchema[LocalDeployers]( "LocalDeployers", map[string]*schema.PropertySchema{ "image": schema.NewPropertySchema( schema.NewAnySchema(), schema.NewDisplayValue( - schema.PointerTo("Local image deployer"), + schema.PointerTo("Local Image deployer"), schema.PointerTo( "Local container environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", ), @@ -174,16 +174,16 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, nil, - //schema.PointerTo("{\"type\":\"docker\"}"), - nil, + schema.PointerTo("{\"type\":\"docker\"}"), + //nil, nil, ), "python": schema.NewPropertySchema( schema.NewAnySchema(), schema.NewDisplayValue( - schema.PointerTo("Local python deployer"), + schema.PointerTo("Local Python deployer"), schema.PointerTo( - "Local python environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", + "Local Python environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", ), nil, ), @@ -192,8 +192,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, nil, - //schema.PointerTo("{\"type\":\"python\"}"), - nil, + schema.PointerTo("{\"type\":\"python\"}"), nil, ), }, From 6f1cf65bbf8cf8e9cdfbbb16c735572037d37fd7 Mon Sep 17 00:00:00 2001 From: mleader Date: Wed, 25 Oct 2023 15:44:03 -0400 Subject: [PATCH 25/54] fix broken tests --- config/load_test.go | 15 ++++++++++----- config/schema.go | 1 + 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/config/load_test.go b/config/load_test.go index 8cf54537..3626b248 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -19,7 +19,8 @@ var configLoadData = map[string]struct { expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: config.LocalDeployers{ - Image: "docker", + Image: map[string]string{"type": "docker"}, + Python: map[string]string{"type": "python"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -35,7 +36,8 @@ log: expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: config.LocalDeployers{ - Image: "docker", + Image: map[string]string{"type": "docker"}, + Python: map[string]string{"type": "python"}, }, Log: log.Config{ Level: log.LevelDebug, @@ -46,12 +48,14 @@ log: "type-kubernetes": { input: ` deployers: - image: kubernetes + image: + type: kubernetes `, expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: config.LocalDeployers{ - Image: "kubernetes", + Image: map[string]string{"type": "kubernetes"}, + Python: map[string]string{"type": "python"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -69,7 +73,8 @@ plugins: "quay.io/arcalot/example-plugin:latest", }, LocalDeployers: config.LocalDeployers{ - Image: "docker", + Image: map[string]string{"type": "docker"}, + Python: map[string]string{"type": "python"}, }, Log: log.Config{ Level: log.LevelInfo, diff --git a/config/schema.go b/config/schema.go index 086908b2..876a7e21 100644 --- a/config/schema.go +++ b/config/schema.go @@ -193,6 +193,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, schema.PointerTo("{\"type\":\"python\"}"), + //schema.PointerTo(`{"type": "python", "pythonPath": "/usr/bin/python"`), nil, ), }, From 0e9e558ce8733d165e483b1d3fd4071a132207ac Mon Sep 17 00:00:00 2001 From: mleader Date: Wed, 25 Oct 2023 17:07:33 -0400 Subject: [PATCH 26/54] fix broken tests --- engine_test.go | 4 +- internal/step/plugin/provider.go | 37 ++++++++++---- internal/step/plugin/provider_test.go | 74 ++++++++++++++++++++------- 3 files changed, 84 insertions(+), 31 deletions(-) diff --git a/engine_test.go b/engine_test.go index 3ff7cc5b..5496171f 100644 --- a/engine_test.go +++ b/engine_test.go @@ -16,7 +16,9 @@ import ( func TestEngineWorkflow_ParseVersion(t *testing.T) { _, err := engine.SupportedVersion("v0.1.0") assert.NoError(t, err) - _, err = engine.SupportedVersion("v0.11.0") + + // test unsupported version + _, err = engine.SupportedVersion("v0.1000.0") assert.Error(t, err) } diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 71fc0354..8d004b25 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -27,18 +27,23 @@ const errorStr = "error" // deployment with the 'type' key. // For more info, see `config/schema.go` func New(logger log.Logger, deployerRegistry registry.Registry, localDeployerConfig any) (step.Provider, error) { - unserializedLocalDeployerConfig, err := deployerRegistry.Schema().Unserialize(localDeployerConfig) - if err != nil { - return nil, fmt.Errorf("failed to load local deployer configuration, please check your Arcaflow configuration file (%w)", err) - } - localDeployer, err := deployerRegistry.Create(unserializedLocalDeployerConfig, logger.WithLabel("source", "deployer")) - if err != nil { - return nil, fmt.Errorf("invalid local deployer configuration, please check your Arcaflow configuration file (%w)", err) + localDeployers := make(map[string]deployer.Connector) + for key, value := range localDeployerConfig.(map[string]any) { + unserializedLocalDeployerConfig, err := deployerRegistry.Schema().Unserialize(value) + if err != nil { + return nil, fmt.Errorf("failed to load local deployer configuration, please check your Arcaflow configuration file (%w)", err) + } + localDeployer, err := deployerRegistry.Create(unserializedLocalDeployerConfig, logger.WithLabel("source", "deployer")) + if err != nil { + return nil, fmt.Errorf("invalid local deployer configuration, please check your Arcaflow configuration file (%w)", err) + } + localDeployers[key] = localDeployer } + return &pluginProvider{ logger: logger.WithLabel("source", "plugin-provider"), deployerRegistry: deployerRegistry, - localDeployer: localDeployer, + localDeployers: localDeployers, }, nil } @@ -46,9 +51,15 @@ func (p *pluginProvider) Kind() string { return "plugin" } +type LocalDeployers struct { + Image deployer.Connector + Python deployer.Connector +} + type pluginProvider struct { deployerRegistry registry.Registry localDeployer deployer.Connector + localDeployers map[string]deployer.Connector logger log.Logger } @@ -188,12 +199,16 @@ func (p *pluginProvider) Lifecycle() step.Lifecycle[step.LifecycleStage] { // LoadSchema deploys the plugin, connects to the plugin's ATP server, loads its schema, then // returns a runnableStep struct. Not to be confused with the runningStep struct. func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) (step.RunnableStep, error) { - image := inputs["plugin"].(string) + image := inputs["plugin"].(map[string]string) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - plugin, err := p.localDeployer.Deploy(ctx, image) + //src, ok := image["image"] + //_, ok := p.localDeployers[] + p.localDeployer = p.localDeployers[image["type"]] + + plugin, err := p.localDeployer.Deploy(ctx, image["src"]) if err != nil { cancel() return nil, fmt.Errorf("failed to deploy plugin from image %s (%w)", image, err) @@ -218,7 +233,7 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) return &runnableStep{ schemas: *s, logger: p.logger, - image: image, + image: image["src"], deployerRegistry: p.deployerRegistry, localDeployer: p.localDeployer, }, nil diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 2db5f39b..c3719469 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -112,7 +112,7 @@ func (s *stageChangeHandler) OnStepComplete( func TestProvider_Utility(t *testing.T) { workflowDeployerCfg := map[string]any{ - "type": "test-impl", + "image": map[string]any{"type": "test-impl"}, } plp, err := plugin.New( @@ -133,7 +133,9 @@ func TestProvider_Utility(t *testing.T) { assert.NotNil(t, plp.Lifecycle()) stepSchema := map[string]any{ - "plugin": "simulation", + "plugin": map[string]string{ + "src": "simulation", + "type": "image"}, } byteSchema := map[string][]byte{} @@ -161,7 +163,7 @@ func TestProvider_HappyError(t *testing.T) { }, ) workflowDeployerCfg := map[string]any{ - "type": "test-impl", + "image": map[string]any{"type": "test-impl"}, } deployerRegistry := deployer_registry.New( @@ -181,8 +183,14 @@ func TestProvider_HappyError(t *testing.T) { ) assert.NoError(t, err) - runnable, err := plp.LoadSchema( - map[string]any{"plugin": "simulation"}, map[string][]byte{}) + stepSchema := map[string]any{ + "plugin": map[string]string{ + "src": "simulation", + "type": "image"}, + } + byteSchema := map[string][]byte{} + + runnable, err := plp.LoadSchema(stepSchema, byteSchema) assert.NoError(t, err) handler := &stageChangeHandler{ @@ -271,7 +279,7 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { }, ) workflowDeployerCfg := map[string]any{ - "type": "test-impl", + "image": map[string]any{"type": "test-impl"}, } deployerRegistry := deployer_registry.New( @@ -284,8 +292,14 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { ) assert.NoError(t, err) - runnable, err := plp.LoadSchema( - map[string]any{"plugin": "simulation"}, map[string][]byte{}) + stepSchema := map[string]any{ + "plugin": map[string]string{ + "src": "simulation", + "type": "image"}, + } + byteSchema := map[string][]byte{} + + runnable, err := plp.LoadSchema(stepSchema, byteSchema) assert.NoError(t, err) assert.NotNil(t, runnable) @@ -326,10 +340,16 @@ func TestProvider_DeployFail(t *testing.T) { ) deployTimeMs := 20 + //workflowDeployerCfg := map[string]any{ + // "type": "test-impl", + // "deploy_time": deployTimeMs, + // "deploy_succeed": true, + //} + workflowDeployerCfg := map[string]any{ - "type": "test-impl", - "deploy_time": deployTimeMs, - "deploy_succeed": true, + "image": map[string]any{"type": "test-impl"}, + //"deploy_time": deployTimeMs, + //"deploy_succeed": true, } deployerRegistry := deployer_registry.New( @@ -342,8 +362,14 @@ func TestProvider_DeployFail(t *testing.T) { ) assert.NoError(t, err) - runnable, err := plp.LoadSchema( - map[string]any{"plugin": "simulation"}, map[string][]byte{}) + stepSchema := map[string]any{ + "plugin": map[string]string{ + "src": "simulation", + "type": "image"}, + } + byteSchema := map[string][]byte{} + + runnable, err := plp.LoadSchema(stepSchema, byteSchema) assert.NoError(t, err) handler := &deployFailStageChangeHandler{ @@ -391,10 +417,15 @@ func TestProvider_StartFail(t *testing.T) { }, ) deployTimeMs := 20 + //workflowDeployerCfg := map[string]any{ + // "type": "test-impl", + // "deploy_time": deployTimeMs, + // "deploy_succeed": true, + //} workflowDeployerCfg := map[string]any{ - "type": "test-impl", - "deploy_time": deployTimeMs, - "deploy_succeed": true, + "image": map[string]any{"type": "test-impl"}, + //"deploy_time": deployTimeMs, + //"deploy_succeed": true, } plp, err := plugin.New( @@ -405,9 +436,14 @@ func TestProvider_StartFail(t *testing.T) { ) assert.NoError(t, err) - runnable, err := plp.LoadSchema( - map[string]any{"plugin": "simulation"}, - map[string][]byte{}) + stepSchema := map[string]any{ + "plugin": map[string]string{ + "src": "simulation", + "type": "image"}, + } + byteSchema := map[string][]byte{} + + runnable, err := plp.LoadSchema(stepSchema, byteSchema) assert.NoError(t, err) handler := &startFailStageChangeHandler{ From 5d9d4f6225255d4e75e5943e62ac2bcc6beb872b Mon Sep 17 00:00:00 2001 From: mleader Date: Thu, 26 Oct 2023 11:02:54 -0400 Subject: [PATCH 27/54] update plugin schema with src and type keys --- internal/step/plugin/provider.go | 78 +++++++++++++++++++++++++-- internal/step/plugin/provider_test.go | 3 +- 2 files changed, 76 insertions(+), 5 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 8d004b25..c90f1434 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -66,13 +66,83 @@ type pluginProvider struct { func (p *pluginProvider) Register(_ step.Registry) { } +var DeploymentTypes = map[string]struct{}{ + "image": struct{}{}, + "python": struct{}{}, +} + +func KeysString(m map[string]struct{}) string { + keys := make([]string, 0, len(m)) + for k := range m { + keys = append(keys, k) + } + return "[" + strings.Join(keys, ", ") + "]" +} + +type Plugin struct { + Src string + Type string +} + +//func (p *pluginProvider) ProviderSchema() *schema.TypedScopeSchema[*Plugin] { +// return schema.NewTypedScopeSchema[*Plugin]( +// schema.NewStructMappedObjectSchema[*Plugin]( +// "Plugin", +// map[string]*schema.PropertySchema{ +// "src": schema.NewPropertySchema( +// schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), +// schema.NewDisplayValue( +// schema.PointerTo("Source"), +// schema.PointerTo("Source file to be executed."), nil), +// true, +// nil, +// nil, +// nil, +// nil, +// []string{"\"quay.io/arcaflow/example-plugin:latest\""}, +// ), +// "type": schema.NewPropertySchema( +// schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), +// schema.NewDisplayValue( +// schema.PointerTo("Type"), +// schema.PointerTo( +// fmt.Sprintf("Deployment type %s", +// fmt.Sprintf(KeysString(DeploymentTypes)))), +// nil, +// ), +// true, +// nil, +// nil, +// nil, +// nil, +// []string{"docker"}, +// ), +// }, +// ), +// ) +//} + func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { return map[string]*schema.PropertySchema{ - "plugin": schema.NewPropertySchema( + "src": schema.NewPropertySchema( + schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), + schema.NewDisplayValue( + schema.PointerTo("Source"), + schema.PointerTo("Source file to be executed."), nil), + true, + nil, + nil, + nil, + nil, + []string{"\"quay.io/arcaflow/example-plugin:latest\""}, + ), + "type": schema.NewPropertySchema( schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), schema.NewDisplayValue( - schema.PointerTo("Plugin"), - schema.PointerTo("Plugin container image to run. This image must be an Arcaflow-compatible container."), + schema.PointerTo("Type"), + schema.PointerTo( + fmt.Sprintf("Deployment type %s", + fmt.Sprintf(KeysString(DeploymentTypes)))), nil, ), true, @@ -80,7 +150,7 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { nil, nil, nil, - []string{"\"quay.io/arcaflow/example-plugin:latest\""}, + []string{"docker"}, ), } } diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index c3719469..75fdd5d2 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -135,7 +135,8 @@ func TestProvider_Utility(t *testing.T) { stepSchema := map[string]any{ "plugin": map[string]string{ "src": "simulation", - "type": "image"}, + "type": "image", + }, } byteSchema := map[string][]byte{} From c09b7ba43a3f8abf0b7304a58a84c029e2898e1b Mon Sep 17 00:00:00 2001 From: mleader Date: Fri, 27 Oct 2023 10:52:12 -0400 Subject: [PATCH 28/54] remove duplicated test --- internal/step/plugin/provider_test.go | 63 --------------------------- 1 file changed, 63 deletions(-) diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 5a492987..ac180f99 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -325,69 +325,6 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { } } -func TestProvider_DeployFail(t *testing.T) { - logConfig := log.Config{ - Level: log.LevelError, - Destination: log.DestinationStdout, - } - logger := log.New( - log.Config{ - Level: log.LevelError, - Destination: log.DestinationStdout, - }, - ) - workflowDeployerCfg := map[string]any{ - "image": map[string]any{"type": "test-impl"}, - } - - deployerRegistry := deployer_registry.New( - deployer.Any(testdeployer.NewFactory())) - - plp, err := plugin.New( - logger, - deployerRegistry, - workflowDeployerCfg, - ) - assert.NoError(t, err) - - stepSchema := map[string]any{ - "plugin": map[string]string{ - "src": "simulation", - "type": "image"}, - } - byteSchema := map[string][]byte{} - - runnable, err := plp.LoadSchema(stepSchema, byteSchema) - assert.NoError(t, err) - assert.NotNil(t, runnable) - - waitLifecycle, err := runnable.Lifecycle(map[string]any{"step": "wait"}) - assert.NoError(t, err) - // Verify that the expected lifecycle stage is there, then verify that cancel is disabled. - waitCancelledStageIDIndex := assert.SliceContainsExtractor(t, - func(schema step.LifecycleStageWithSchema) string { - return schema.ID - }, string(plugin.StageIDCancelled), waitLifecycle.Stages) - waitStageIDCancelled := waitLifecycle.Stages[waitCancelledStageIDIndex] - waitStopIfSchema := assert.MapContainsKey(t, "stop_if", waitStageIDCancelled.InputSchema) - if waitStopIfSchema.Disabled { - t.Fatalf("step wait's wait_for schema is disabled when the cancel signal is present.") - } - - helloLifecycle, err := runnable.Lifecycle(map[string]any{"step": "hello"}) - assert.NoError(t, err) - // Verify that the expected lifecycle stage is there, then verify that cancel is disabled. - helloCancelledStageIDIndex := assert.SliceContainsExtractor(t, - func(schema step.LifecycleStageWithSchema) string { - return schema.ID - }, string(plugin.StageIDCancelled), helloLifecycle.Stages) - helloStageIDCancelled := helloLifecycle.Stages[helloCancelledStageIDIndex] - helloStopIfSchema := assert.MapContainsKey(t, "stop_if", helloStageIDCancelled.InputSchema) - if !helloStopIfSchema.Disabled { - t.Fatalf("step hello's stop_if schema is not disabled when the cancel signal is not present.") - } -} - func TestProvider_DeployFail(t *testing.T) { logConfig := log.Config{ Level: log.LevelError, From b136c2a56205c4122827b140b8b778610815fbbe Mon Sep 17 00:00:00 2001 From: mleader Date: Fri, 27 Oct 2023 11:06:25 -0400 Subject: [PATCH 29/54] add multiple deployer test --- internal/step/plugin/provider_test.go | 92 ++++++++++++++++++++------- 1 file changed, 70 insertions(+), 22 deletions(-) diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index ac180f99..efd4757c 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -110,6 +110,60 @@ func (s *stageChangeHandler) OnStepComplete( s.message <- message } +func TestProvider_MultipleDeployers(t *testing.T) { + logger := log.New( + log.Config{ + Level: log.LevelError, + Destination: log.DestinationStdout, + }, + ) + deployerRegistry := deployer_registry.New( + deployer.Any(testdeployer.NewFactory())) + deployTimeMs := 20 + workflowDeployerCfg := map[string]any{ + "image": map[string]any{ + "type": "test-impl", + "deploy_time": deployTimeMs, + "deploy_succeed": true, + }, + "python": map[string]any{"type": "test-impl"}, + } + + plp, err := plugin.New( + logger, + deployerRegistry, + workflowDeployerCfg, + ) + assert.NoError(t, err) + assert.Equals(t, plp.Kind(), "plugin") + assert.NotNil(t, plp.ProviderSchema()) + assert.NotNil(t, plp.RunProperties()) + assert.NotNil(t, plp.Lifecycle()) + + stepSchema := map[string]any{ + "plugin": map[string]string{ + "src": "simulation", + "type": "image", + }, + } + byteSchema := map[string][]byte{} + + runnable, err := plp.LoadSchema(stepSchema, byteSchema) + assert.NoError(t, err) + + assert.NotNil(t, runnable.RunSchema()) + + _, err = runnable.Lifecycle(map[string]any{"step": "wait"}) + assert.NoError(t, err) + + _, err = runnable.Lifecycle(map[string]any{"step": "hello"}) + assert.NoError(t, err) + + // There is more than one step, so no specified one will cause an error. + _, err = runnable.Lifecycle(map[string]any{"step": nil}) + assert.Error(t, err) +} + func TestProvider_Utility(t *testing.T) { workflowDeployerCfg := map[string]any{ "image": map[string]any{"type": "test-impl"}, @@ -326,30 +380,21 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { } func TestProvider_DeployFail(t *testing.T) { - logConfig := log.Config{ - Level: log.LevelError, - Destination: log.DestinationStdout, - } logger := log.New( - logConfig, + log.Config{ + Level: log.LevelError, + Destination: log.DestinationStdout, + }, ) - + deployerRegistry := deployer_registry.New( + deployer.Any(testdeployer.NewFactory())) deployTimeMs := 20 - //workflowDeployerCfg := map[string]any{ - // "type": "test-impl", - // "deploy_time": deployTimeMs, - // "deploy_succeed": true, - //} - workflowDeployerCfg := map[string]any{ - "image": map[string]any{"type": "test-impl"}, - //"deploy_time": deployTimeMs, - //"deploy_succeed": true, + "image": map[string]any{"type": "test-impl"}, + "deploy_time": deployTimeMs, + "deploy_succeed": true, } - deployerRegistry := deployer_registry.New( - deployer.Any(testdeployer.NewFactory())) - plp, err := plugin.New( logger, deployerRegistry, @@ -377,10 +422,13 @@ func TestProvider_DeployFail(t *testing.T) { assert.NoError(t, running.ProvideStageInput( string(plugin.StageIDDeploy), - map[string]any{"deploy": map[string]any{ - "type": "test-impl", - "deploy_succeed": false, - "deploy_time": deployTimeMs}}, + map[string]any{ + "deploy": map[string]any{ + "type": "test-impl", + //"deploy_succeed": false, + //"deploy_time": deployTimeMs, + }, + }, )) waitTimeMs := 50 From 2e8b44d33bcd76ec7ade74ec41b08b9d497c5fce Mon Sep 17 00:00:00 2001 From: mleader Date: Fri, 27 Oct 2023 13:19:25 -0400 Subject: [PATCH 30/54] investigate workflow config deployer schema --- internal/step/plugin/provider.go | 38 ------------ internal/step/plugin/provider_test.go | 88 +++++++++++++++------------ 2 files changed, 48 insertions(+), 78 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index c90f1434..35e3404a 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -84,44 +84,6 @@ type Plugin struct { Type string } -//func (p *pluginProvider) ProviderSchema() *schema.TypedScopeSchema[*Plugin] { -// return schema.NewTypedScopeSchema[*Plugin]( -// schema.NewStructMappedObjectSchema[*Plugin]( -// "Plugin", -// map[string]*schema.PropertySchema{ -// "src": schema.NewPropertySchema( -// schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), -// schema.NewDisplayValue( -// schema.PointerTo("Source"), -// schema.PointerTo("Source file to be executed."), nil), -// true, -// nil, -// nil, -// nil, -// nil, -// []string{"\"quay.io/arcaflow/example-plugin:latest\""}, -// ), -// "type": schema.NewPropertySchema( -// schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), -// schema.NewDisplayValue( -// schema.PointerTo("Type"), -// schema.PointerTo( -// fmt.Sprintf("Deployment type %s", -// fmt.Sprintf(KeysString(DeploymentTypes)))), -// nil, -// ), -// true, -// nil, -// nil, -// nil, -// nil, -// []string{"docker"}, -// ), -// }, -// ), -// ) -//} - func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { return map[string]*schema.PropertySchema{ "src": schema.NewPropertySchema( diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index efd4757c..1527f91f 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -126,7 +126,9 @@ func TestProvider_MultipleDeployers(t *testing.T) { "deploy_time": deployTimeMs, "deploy_succeed": true, }, - "python": map[string]any{"type": "test-impl"}, + "python": map[string]any{ + "type": "test-impl", + }, } plp, err := plugin.New( @@ -395,6 +397,12 @@ func TestProvider_DeployFail(t *testing.T) { "deploy_succeed": true, } + _ := map[string]any{ + "type": "test-impl", + "deploy_time": deployTimeMs, + "deploy_succeed": true, + } + plp, err := plugin.New( logger, deployerRegistry, @@ -409,47 +417,47 @@ func TestProvider_DeployFail(t *testing.T) { } byteSchema := map[string][]byte{} - runnable, err := plp.LoadSchema(stepSchema, byteSchema) + _, err = plp.LoadSchema(stepSchema, byteSchema) assert.NoError(t, err) - handler := &deployFailStageChangeHandler{ - message: make(chan string), - } - - // wait step - running, err := runnable.Start(map[string]any{"step": "wait"}, t.Name(), handler) - assert.NoError(t, err) - - assert.NoError(t, running.ProvideStageInput( - string(plugin.StageIDDeploy), - map[string]any{ - "deploy": map[string]any{ - "type": "test-impl", - //"deploy_succeed": false, - //"deploy_time": deployTimeMs, - }, - }, - )) - - waitTimeMs := 50 - assert.NoError(t, running.ProvideStageInput( - string(plugin.StageIDStarting), - map[string]any{"input": map[string]any{"wait_time_ms": waitTimeMs}}, - )) - - message := <-handler.message - assert.Equals(t, - message, - fmt.Sprintf("intentional deployment fail after %d ms", deployTimeMs)) - - assert.Equals(t, string(running.State()), - string(step.RunningStepStateFinished)) - - assert.Equals(t, running.CurrentStage(), string(plugin.StageIDDeployFailed)) - - t.Cleanup(func() { - assert.NoError(t, running.Close()) - }) + //handler := &deployFailStageChangeHandler{ + // message: make(chan string), + //} + // + //// wait step + //running, err := runnable.Start(map[string]any{"step": "wait"}, t.Name(), handler) + //assert.NoError(t, err) + // + ////assert.NoError(t, running.ProvideStageInput( + //// string(plugin.StageIDDeploy), + //// map[string]any{ + //// "deploy": map[string]any{ + //// "type": "test-impl", + //// //"deploy_succeed": false, + //// //"deploy_time": deployTimeMs, + //// }, + //// }, + ////)) + //// + ////waitTimeMs := 50 + ////assert.NoError(t, running.ProvideStageInput( + //// string(plugin.StageIDStarting), + //// map[string]any{"input": map[string]any{"wait_time_ms": waitTimeMs}}, + ////)) + //// + ////message := <-handler.message + ////assert.Equals(t, + //// message, + //// fmt.Sprintf("intentional deployment fail after %d ms", deployTimeMs)) + //// + ////assert.Equals(t, string(running.State()), + //// string(step.RunningStepStateFinished)) + //// + ////assert.Equals(t, running.CurrentStage(), string(plugin.StageIDDeployFailed)) + //// + ////t.Cleanup(func() { + //// assert.NoError(t, running.Close()) + ////}) } func TestProvider_StartFail(t *testing.T) { From 0660e38c9e92a5adc3e6257e3235e9523756015a Mon Sep 17 00:00:00 2001 From: mleader Date: Fri, 27 Oct 2023 13:30:30 -0400 Subject: [PATCH 31/54] refactor deployer discriminator to deployer_id --- config/load_test.go | 16 +++++++-------- internal/step/plugin/provider.go | 2 +- internal/step/plugin/provider_test.go | 28 +++++++++++++-------------- workflow/executor_test.go | 2 +- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/config/load_test.go b/config/load_test.go index 3626b248..05586608 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -19,8 +19,8 @@ var configLoadData = map[string]struct { expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"type": "docker"}, - Python: map[string]string{"type": "python"}, + Image: map[string]string{"deployer_id": "docker"}, + Python: map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -36,8 +36,8 @@ log: expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"type": "docker"}, - Python: map[string]string{"type": "python"}, + Image: map[string]string{"deployer_id": "docker"}, + Python: map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelDebug, @@ -54,8 +54,8 @@ deployers: expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"type": "kubernetes"}, - Python: map[string]string{"type": "python"}, + Image: map[string]string{"deployer_id": "kubernetes"}, + Python: map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -73,8 +73,8 @@ plugins: "quay.io/arcalot/example-plugin:latest", }, LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"type": "docker"}, - Python: map[string]string{"type": "python"}, + Image: map[string]string{"deployer_id": "docker"}, + Python: map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelInfo, diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 35e3404a..2b957acb 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -98,7 +98,7 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { nil, []string{"\"quay.io/arcaflow/example-plugin:latest\""}, ), - "type": schema.NewPropertySchema( + "deployer_id": schema.NewPropertySchema( schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), schema.NewDisplayValue( schema.PointerTo("Type"), diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 1527f91f..34bcb469 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -122,12 +122,12 @@ func TestProvider_MultipleDeployers(t *testing.T) { deployTimeMs := 20 workflowDeployerCfg := map[string]any{ "image": map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", "deploy_time": deployTimeMs, "deploy_succeed": true, }, "python": map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", }, } @@ -144,8 +144,8 @@ func TestProvider_MultipleDeployers(t *testing.T) { stepSchema := map[string]any{ "plugin": map[string]string{ - "src": "simulation", - "type": "image", + "src": "simulation", + "deployer_id": "image", }, } byteSchema := map[string][]byte{} @@ -168,7 +168,7 @@ func TestProvider_MultipleDeployers(t *testing.T) { func TestProvider_Utility(t *testing.T) { workflowDeployerCfg := map[string]any{ - "image": map[string]any{"type": "test-impl"}, + "image": map[string]any{"deployer_id": "test-impl"}, } plp, err := plugin.New( @@ -220,7 +220,7 @@ func TestProvider_HappyError(t *testing.T) { }, ) workflowDeployerCfg := map[string]any{ - "image": map[string]any{"type": "test-impl"}, + "image": map[string]any{"deployer_id": "test-impl"}, } deployerRegistry := deployer_registry.New( @@ -270,14 +270,14 @@ func TestProvider_HappyError(t *testing.T) { assert.Error(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", "deploy_time": "abc"}}, )) assert.NoError(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", "deploy_time": 1}}, )) @@ -285,7 +285,7 @@ func TestProvider_HappyError(t *testing.T) { assert.Error(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", "deploy_time": nil}}, )) @@ -336,7 +336,7 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { }, ) workflowDeployerCfg := map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", } deployerRegistry := deployer_registry.New( @@ -392,13 +392,13 @@ func TestProvider_DeployFail(t *testing.T) { deployer.Any(testdeployer.NewFactory())) deployTimeMs := 20 workflowDeployerCfg := map[string]any{ - "image": map[string]any{"type": "test-impl"}, + "image": map[string]any{"deployer_id": "test-impl"}, "deploy_time": deployTimeMs, "deploy_succeed": true, } _ := map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", "deploy_time": deployTimeMs, "deploy_succeed": true, } @@ -474,7 +474,7 @@ func TestProvider_StartFail(t *testing.T) { // "deploy_succeed": true, //} workflowDeployerCfg := map[string]any{ - "image": map[string]any{"type": "test-impl"}, + "image": map[string]any{"deployer_id": "test-impl"}, //"deploy_time": deployTimeMs, //"deploy_succeed": true, } @@ -508,7 +508,7 @@ func TestProvider_StartFail(t *testing.T) { assert.NoError(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "type": "test-impl", + "deployer_id": "test-impl", "deploy_succeed": true, "deploy_time": deployTimeMs, "disable_plugin_writes": true}}, diff --git a/workflow/executor_test.go b/workflow/executor_test.go index 80fae2c4..67e23752 100644 --- a/workflow/executor_test.go +++ b/workflow/executor_test.go @@ -64,7 +64,7 @@ func NewTestImplStepRegistry( pluginProvider := assert.NoErrorR[step.Provider](t)( plugin.New(logger, deployerRegistry, map[string]interface{}{ - "type": "test-impl", + "deployer_id": "test-impl", "deploy_time": "0", }), ) From 0c4bcc7bb6f29050e02e7b1faa33eae0d4615044 Mon Sep 17 00:00:00 2001 From: mleader Date: Fri, 27 Oct 2023 13:37:42 -0400 Subject: [PATCH 32/54] refactor deployer discriminator field to use deployer_id --- config/load_test.go | 2 +- config/schema.go | 5 ++--- internal/step/plugin/provider_test.go | 6 ------ 3 files changed, 3 insertions(+), 10 deletions(-) diff --git a/config/load_test.go b/config/load_test.go index 05586608..dbe37940 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -49,7 +49,7 @@ log: input: ` deployers: image: - type: kubernetes + deployer_id: kubernetes `, expectedOutput: &config.Config{ TypeHintPlugins: nil, diff --git a/config/schema.go b/config/schema.go index 876a7e21..82e500a2 100644 --- a/config/schema.go +++ b/config/schema.go @@ -174,7 +174,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, nil, - schema.PointerTo("{\"type\":\"docker\"}"), + schema.PointerTo("{\"deployer_id\":\"docker\"}"), //nil, nil, ), @@ -192,8 +192,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, nil, - schema.PointerTo("{\"type\":\"python\"}"), - //schema.PointerTo(`{"type": "python", "pythonPath": "/usr/bin/python"`), + schema.PointerTo("{\"deployer_id\":\"python\"}"), nil, ), }, diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 34bcb469..12bd143c 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -397,12 +397,6 @@ func TestProvider_DeployFail(t *testing.T) { "deploy_succeed": true, } - _ := map[string]any{ - "deployer_id": "test-impl", - "deploy_time": deployTimeMs, - "deploy_succeed": true, - } - plp, err := plugin.New( logger, deployerRegistry, From a8894dd55205b915aa77bddb422a330786889a79 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Fri, 27 Oct 2023 17:46:11 -0400 Subject: [PATCH 33/54] Progress towards implementation of deployment types --- config/config.go | 7 +-- config/load_test.go | 24 ++++---- config/schema.go | 49 +++------------ engine.go | 8 +-- internal/step/plugin/provider.go | 100 ++++++++++++++++++------------- new.go | 7 +-- workflow/workflow_test.go | 65 +++++++++++++++----- 7 files changed, 132 insertions(+), 128 deletions(-) diff --git a/config/config.go b/config/config.go index 0389d69e..3468e719 100644 --- a/config/config.go +++ b/config/config.go @@ -17,14 +17,9 @@ type Config struct { TypeHintPlugins []string `json:"plugins" yaml:"plugins"` // LocalDeployers holds the configuration for executing plugins locally. This deployer is used to obtain the schema // from the plugins before executing them in a remote environment. - LocalDeployers LocalDeployers `json:"deployers" yaml:"deployers"` + LocalDeployers map[string]any `json:"deployers" yaml:"deployers"` // Log configures logging for workflow runs. Log log.Config `json:"log" yaml:"log"` // StepOutputLogging allows logging of step output LoggedOutputConfigs map[string]*StepOutputLogConfig `json:"logged_outputs" yaml:"logged_outputs"` } - -type LocalDeployers struct { - Image any `json:"image" yaml:"image"` - Python any `json:"python" yaml:"python"` -} diff --git a/config/load_test.go b/config/load_test.go index dbe37940..660dc27b 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -18,9 +18,9 @@ var configLoadData = map[string]struct { input: "", expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"deployer_id": "docker"}, - Python: map[string]string{"deployer_id": "python"}, + LocalDeployers: map[string]any{ + "image": map[string]string{"deployer_id": "docker"}, + "python": map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -35,9 +35,9 @@ log: `, expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"deployer_id": "docker"}, - Python: map[string]string{"deployer_id": "python"}, + LocalDeployers: map[string]any{ + "image": map[string]string{"deployer_id": "docker"}, + "python": map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelDebug, @@ -53,9 +53,9 @@ deployers: `, expectedOutput: &config.Config{ TypeHintPlugins: nil, - LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"deployer_id": "kubernetes"}, - Python: map[string]string{"deployer_id": "python"}, + LocalDeployers: map[string]any{ + "image": map[string]string{"deployer_id": "kubernetes"}, + "python": map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -72,9 +72,9 @@ plugins: TypeHintPlugins: []string{ "quay.io/arcalot/example-plugin:latest", }, - LocalDeployers: config.LocalDeployers{ - Image: map[string]string{"deployer_id": "docker"}, - Python: map[string]string{"deployer_id": "python"}, + LocalDeployers: map[string]any{ + "image": map[string]string{"deployer_id": "docker"}, + "python": map[string]string{"deployer_id": "python"}, }, Log: log.Config{ Level: log.LevelInfo, diff --git a/config/schema.go b/config/schema.go index 82e500a2..35dee1b4 100644 --- a/config/schema.go +++ b/config/schema.go @@ -46,7 +46,12 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, ), "deployers": schema.NewPropertySchema( - schema.NewRefSchema("LocalDeployers", nil), + schema.NewMapSchema( + schema.NewStringSchema(nil, nil, nil), + schema.NewAnySchema(), + nil, + nil, + ), schema.NewDisplayValue( schema.PointerTo("Local deployers"), schema.PointerTo( @@ -58,7 +63,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, nil, - nil, + schema.PointerTo(`{"image": {"deployer_id": "docker"}}`), nil, ), "logged_outputs": schema.NewPropertySchema( @@ -157,45 +162,5 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { ), }, ), - schema.NewStructMappedObjectSchema[LocalDeployers]( - "LocalDeployers", - map[string]*schema.PropertySchema{ - "image": schema.NewPropertySchema( - schema.NewAnySchema(), - schema.NewDisplayValue( - schema.PointerTo("Local Image deployer"), - schema.PointerTo( - "Local container environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", - ), - nil, - ), - //true, - false, - nil, - nil, - nil, - schema.PointerTo("{\"deployer_id\":\"docker\"}"), - //nil, - nil, - ), - "python": schema.NewPropertySchema( - schema.NewAnySchema(), - schema.NewDisplayValue( - schema.PointerTo("Local Python deployer"), - schema.PointerTo( - "Local Python environment configuration the workflow engine can use to test-deploy plugins before the workflow execution.", - ), - nil, - ), - //true, - false, - nil, - nil, - nil, - schema.PointerTo("{\"deployer_id\":\"python\"}"), - nil, - ), - }, - ), ) } diff --git a/engine.go b/engine.go index df3dd536..56c484f9 100644 --- a/engine.go +++ b/engine.go @@ -5,7 +5,6 @@ import ( "context" "fmt" log "go.arcalot.io/log/v2" - "go.flow.arcalot.io/deployer/registry" "go.flow.arcalot.io/engine/config" "go.flow.arcalot.io/engine/internal/step" "go.flow.arcalot.io/engine/internal/yaml" @@ -58,10 +57,9 @@ type Workflow interface { } type workflowEngine struct { - logger log.Logger - deployerRegistry registry.Registry - stepRegistry step.Registry - config *config.Config + logger log.Logger + stepRegistry step.Registry + config *config.Config } func (w workflowEngine) RunWorkflow(ctx context.Context, input []byte, workflowContext map[string][]byte, workflowFileName string) (outputID string, outputData any, outputError bool, err error) { diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 2b957acb..98f4a903 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -26,18 +26,32 @@ const errorStr = "error" // deployers. Most importantly it specifies which deployer is used for this // deployment with the 'type' key. // For more info, see `config/schema.go` -func New(logger log.Logger, deployerRegistry registry.Registry, localDeployerConfig any) (step.Provider, error) { - localDeployers := make(map[string]deployer.Connector) - for key, value := range localDeployerConfig.(map[string]any) { - unserializedLocalDeployerConfig, err := deployerRegistry.Schema().Unserialize(value) - if err != nil { - return nil, fmt.Errorf("failed to load local deployer configuration, please check your Arcaflow configuration file (%w)", err) +func New(logger log.Logger, deployerRegistry registry.Registry, localDeployerConfigs map[string]any) (step.Provider, error) { + localDeployers := make(map[deployer.DeploymentType]deployer.Connector) + for _, deploymentType := range deployerRegistry.DeploymentTypes() { + _, configHasType := localDeployerConfigs[string(deploymentType)] + if !configHasType { + logger.Infof("Config does not have config for deployment type '%s'. This type will not be usable.", deploymentType) + continue + } + localDeployerConfig, configPresent := localDeployerConfigs[string(deploymentType)] + var unserializedLocalDeployerConfig any + var err error + if configPresent { + unserializedLocalDeployerConfig, err = deployerRegistry.DeployConfigSchema(deploymentType).Unserialize(localDeployerConfig) + if err != nil { + return nil, fmt.Errorf("failed to load local deployer configuration, please check your Arcaflow configuration file (%w)", err) + } + } else { + logger.Warningf("No deployer configuration for type '%s' (%w). The workflow will fail if this type is needed", + deploymentType, err) } - localDeployer, err := deployerRegistry.Create(unserializedLocalDeployerConfig, logger.WithLabel("source", "deployer")) + + localDeployer, err := deployerRegistry.Create(deploymentType, unserializedLocalDeployerConfig, logger.WithLabel("source", "deployer")) if err != nil { return nil, fmt.Errorf("invalid local deployer configuration, please check your Arcaflow configuration file (%w)", err) } - localDeployers[key] = localDeployer + localDeployers[deploymentType] = localDeployer } return &pluginProvider{ @@ -51,15 +65,9 @@ func (p *pluginProvider) Kind() string { return "plugin" } -type LocalDeployers struct { - Image deployer.Connector - Python deployer.Connector -} - type pluginProvider struct { deployerRegistry registry.Registry - localDeployer deployer.Connector - localDeployers map[string]deployer.Connector + localDeployers map[deployer.DeploymentType]deployer.Connector logger log.Logger } @@ -98,7 +106,7 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { nil, []string{"\"quay.io/arcaflow/example-plugin:latest\""}, ), - "deployer_id": schema.NewPropertySchema( + "deployment_type": schema.NewPropertySchema( schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), schema.NewDisplayValue( schema.PointerTo("Type"), @@ -231,19 +239,20 @@ func (p *pluginProvider) Lifecycle() step.Lifecycle[step.LifecycleStage] { // LoadSchema deploys the plugin, connects to the plugin's ATP server, loads its schema, then // returns a runnableStep struct. Not to be confused with the runningStep struct. func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) (step.RunnableStep, error) { - image := inputs["plugin"].(map[string]string) + pluginSrcInput := inputs["plugin"].(map[string]string) + requestedDeploymentType := deployer.DeploymentType(pluginSrcInput["deployment_type"]) + pluginSource := pluginSrcInput["src"] ctx, cancel := context.WithCancel(context.Background()) defer cancel() - //src, ok := image["image"] - //_, ok := p.localDeployers[] - p.localDeployer = p.localDeployers[image["type"]] + applicableLocalDeployer := p.localDeployers[requestedDeploymentType] - plugin, err := p.localDeployer.Deploy(ctx, image["src"]) + plugin, err := applicableLocalDeployer.Deploy(ctx, pluginSource) if err != nil { cancel() - return nil, fmt.Errorf("failed to deploy plugin from image %s (%w)", image, err) + return nil, fmt.Errorf("failed to deploy plugin of deployment type '%s' with source '%s' (%w)", + requestedDeploymentType, pluginSource, err) } // Set up the ATP connection transport := atp.NewClientWithLogger(plugin, p.logger) @@ -251,28 +260,30 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) s, err := transport.ReadSchema() if err != nil { cancel() - return nil, fmt.Errorf("failed to read plugin schema from %s (%w)", image, err) + return nil, fmt.Errorf("failed to read plugin schema from '%s' (%w)", pluginSource, err) } // Tell the server that the client is done if err := transport.Close(); err != nil { - return nil, fmt.Errorf("failed to instruct client to shut down image %s (%w)", image, err) + return nil, fmt.Errorf("failed to instruct client to shut down plugin from source '%s' (%w)", pluginSource, err) } // Shut down the plugin. if err := plugin.Close(); err != nil { - return nil, fmt.Errorf("failed to shut down local plugin from %s (%w)", image, err) + return nil, fmt.Errorf("failed to shut down local plugin from '%s' (%w)", pluginSource, err) } return &runnableStep{ schemas: *s, logger: p.logger, - image: image["src"], + deploymentType: requestedDeploymentType, + source: pluginSource, deployerRegistry: p.deployerRegistry, - localDeployer: p.localDeployer, + localDeployer: applicableLocalDeployer, }, nil } type runnableStep struct { - image string + deploymentType deployer.DeploymentType + source string deployerRegistry registry.Registry logger log.Logger schemas schema.SchemaSchema @@ -309,7 +320,7 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st steps := r.schemas.Steps() if stepID == "" { if len(steps) != 1 { - return result, fmt.Errorf("the 'step' parameter is required for the '%s' plugin", r.image) + return result, fmt.Errorf("the 'step' parameter is required for the '%s' plugin", r.source) } for possibleStepID := range steps { stepID = possibleStepID @@ -317,7 +328,7 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st } stepSchema, ok := r.schemas.Steps()[stepID] if !ok { - return result, fmt.Errorf("the step '%s' does not exist in the '%s' plugin", stepID, r.image) + return result, fmt.Errorf("the step '%s' does not exist in the '%s' plugin", stepID, r.source) } stopIfProperty := schema.NewPropertySchema( @@ -338,11 +349,11 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st cancelSignal := stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] if cancelSignal == nil { // Not present - stopIfProperty.Disable(fmt.Sprintf("Cancel signal with ID '%s' is not present in plugin image '%s', step '%s'. Signal handler IDs present: %v", - plugin.CancellationSignalSchema.ID(), r.image, stepID, reflect.ValueOf(stepSchema.SignalHandlers()).MapKeys())) + stopIfProperty.Disable(fmt.Sprintf("Cancel signal with ID '%s' is not present in plugin '%s', step '%s'. Signal handler IDs present: %v", + plugin.CancellationSignalSchema.ID(), r.source, stepID, reflect.ValueOf(stepSchema.SignalHandlers()).MapKeys())) } else if err := plugin.CancellationSignalSchema.DataSchemaValue.ValidateCompatibility(cancelSignal.DataSchemaValue); err != nil { // Present but incompatible - stopIfProperty.Disable(fmt.Sprintf("Cancel signal invalid schema in plugin image '%s', step '%s' (%s)", r.image, stepID, err)) + stopIfProperty.Disable(fmt.Sprintf("Cancel signal invalid schema in plugin '%s', step '%s' (%s)", r.source, stepID, err)) } return step.Lifecycle[step.LifecycleStageWithSchema]{ @@ -352,7 +363,7 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st LifecycleStage: deployingLifecycleStage, InputSchema: map[string]*schema.PropertySchema{ "deploy": schema.NewPropertySchema( - r.deployerRegistry.Schema(), + r.deployerRegistry.DeployConfigSchema(r.deploymentType), schema.NewDisplayValue( schema.PointerTo("Deployment configuration"), schema.PointerTo( @@ -491,8 +502,8 @@ func (r *runnableStep) Start(input map[string]any, runID string, stageChangeHand i++ } return nil, fmt.Errorf( - "the %s plugin declares more than one possible step, please provide the step name (one of: %s)", - r.image, + "the '%s' plugin declares more than one possible step, please provide the step name (one of: %s)", + r.source, strings.Join(stepNames, ", "), ) } @@ -503,8 +514,8 @@ func (r *runnableStep) Start(input map[string]any, runID string, stageChangeHand stepSchema, ok := steps[stepID] if !ok { return nil, fmt.Errorf( - "plugin %s does not have a step named %s", - r.image, + "plugin '%s' does not have a step named %s", + r.source, stepID, ) } @@ -522,7 +533,8 @@ func (r *runnableStep) Start(input map[string]any, runID string, stageChangeHand deployInput: make(chan any, 1), runInput: make(chan any, 1), logger: r.logger, - image: r.image, + deploymentType: r.deploymentType, + source: r.source, pluginStepID: stepID, state: step.RunningStepStateStarting, localDeployer: r.localDeployer, @@ -553,7 +565,8 @@ type runningStep struct { logger log.Logger currentStage StageID runID string // The ID associated with this execution (the workflow step ID) - image string + deploymentType deployer.DeploymentType + source string pluginStepID string // The ID of the step in the plugin state step.RunningStepState useLocalDeployer bool @@ -621,7 +634,7 @@ func (r *runningStep) provideDeployInput(input map[string]any) error { var unserializedDeployerConfig any var err error if input["deploy"] != nil { - unserializedDeployerConfig, err = r.deployerRegistry.Schema().Unserialize(input["deploy"]) + unserializedDeployerConfig, err = r.deployerRegistry.DeployConfigSchema(r.deploymentType).Unserialize(input["deploy"]) if err != nil { return fmt.Errorf("invalid deployment information (%w)", err) } @@ -827,12 +840,13 @@ func (r *runningStep) deployStage() (deployer.Plugin, error) { var stepDeployer = r.localDeployer if !useLocalDeployer { var err error - stepDeployer, err = r.deployerRegistry.Create(deployerConfig, r.logger.WithLabel("source", "deployer")) + stepDeployer, err = r.deployerRegistry.Create(r.deploymentType, deployerConfig, + r.logger.WithLabel("source", "deployer")) if err != nil { return nil, err } } - container, err := stepDeployer.Deploy(r.ctx, r.image) + container, err := stepDeployer.Deploy(r.ctx, r.source) if err != nil { return nil, err } diff --git a/new.go b/new.go index a4b0bebc..374937d0 100644 --- a/new.go +++ b/new.go @@ -20,9 +20,8 @@ func New( return nil, err } return &workflowEngine{ - logger: logger, - config: config, - stepRegistry: stepRegistry, - deployerRegistry: DefaultDeployerRegistry, + logger: logger, + config: config, + stepRegistry: stepRegistry, }, nil } diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index f7f56254..0e060dba 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -55,13 +55,17 @@ input: properties: {} steps: long_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 2000 stop_if: !expr $.steps.short_wait.outputs short_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: # It needs to be long enough for it to ensure that long_wait is in a running state. @@ -101,18 +105,24 @@ steps: # This one needs to run longer than the total time expected of all the other steps, with # a large enough difference to prevent timing errors breaking the test. end_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 80 # Delay needs to be delayed long enough to ensure that last_step isn't running when it's cancelled by short_wait delay: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 50 last_step: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 0 @@ -121,7 +131,9 @@ steps: # You can verify that this test works by commenting out this line. It should fail. stop_if: !expr $.steps.short_wait.outputs short_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: # End the test quickly. @@ -167,12 +179,16 @@ steps: # This one needs to run longer than the total time expected of all the other steps, with # a large enough difference to prevent timing errors breaking the test. end_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 100 step_to_cancel: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 0 @@ -183,7 +199,9 @@ steps: type: "test-impl" deploy_time: 50 # 50 ms short_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: # End the test quickly. @@ -227,7 +245,9 @@ input: properties: {} steps: wait_1: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 0 @@ -256,13 +276,17 @@ input: properties: {} steps: first_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: # Note: 5ms left only a 2.5ms margin for error. 10ms left almost 6ms. So 10ms min is recommended. wait_time_ms: 10 second_wait: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 10 @@ -332,7 +356,9 @@ input: properties: {} steps: wait_1: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 0 @@ -341,7 +367,9 @@ steps: #deploy_time: 20000 # 10 ms deploy_succeed: false wait_2: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait wait_for: !expr $.steps.wait_1.outputs.success input: @@ -371,12 +399,17 @@ input: properties: {} steps: wait_1: - plugin: "n/a" + plugin: + src: "n/a" + type: "builtin" step: wait input: wait_time_ms: 0 wait_2: - plugin: "n/a" + + plugin: + src: "n/a" + type: "builtin" step: wait # No stop_if, so this shouldn't happen. wait_for: !expr $.steps.wait_1.outputs.cancelled_early From a69ea86e9a77c6298568bec5d2ccc1e04c52625b Mon Sep 17 00:00:00 2001 From: mleader Date: Sat, 28 Oct 2023 16:21:17 -0400 Subject: [PATCH 34/54] getting the static type of an object comes out in golang 1.22 --- new.go | 95 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 94 insertions(+), 1 deletion(-) diff --git a/new.go b/new.go index 374937d0..de81992f 100644 --- a/new.go +++ b/new.go @@ -1,8 +1,15 @@ package engine import ( + "fmt" log "go.arcalot.io/log/v2" + "go.flow.arcalot.io/deployer" + "go.flow.arcalot.io/deployer/registry" + docker "go.flow.arcalot.io/dockerdeployer" "go.flow.arcalot.io/engine/config" + kubernetes "go.flow.arcalot.io/kubernetesdeployer" + podman "go.flow.arcalot.io/podmandeployer" + python "go.flow.arcalot.io/pythondeployer" ) // New creates a new workflow engine with the provided configuration. The passed deployerRegistry is responsible for @@ -11,9 +18,20 @@ func New( config *config.Config, ) (WorkflowEngine, error) { logger := log.New(config.Log) + // TODO: Create new registry function + // need to build/create new deployer registry based on + // deployer_ids in config.LocalDeployers + // use registry.Create and its reflection as inspo + //for key, value := range config.LocalDeployers { + // + //} + reg, err := BuildRegistry(config.LocalDeployers) + if err != nil { + return nil, err + } stepRegistry, err := NewDefaultStepRegistry( logger, - DefaultDeployerRegistry, + reg, config, ) if err != nil { @@ -25,3 +43,78 @@ func New( stepRegistry: stepRegistry, }, nil } + +//var DefaultDeployerRegistry = registry.New( +// deployer.Any(docker.NewFactory()), +// deployer.Any(kubernetes.NewFactory()), +// deployer.Any(podman.NewFactory()), +// deployer.Any(python.NewFactory()), +//) + +type cfg struct{} + +func BuildRegistry(config map[string]any) (registry.Registry, error) { + if config == nil { + return nil, fmt.Errorf("the deployer configuration cannot be nil") + } + + //for _,v := range DefaultDeployerRegistry { + //fmt.Printf("%v\n" v) + //} + + factories := make([]deployer.AnyConnectorFactory, 0) + for deploymentType, value := range config { + v2 := make(map[string]any) + for k, v := range value.(map[any]any) { + v2[k.(string)] = v + } + schemas := DefaultDeployerRegistry.DeployConfigSchema(deployer.DeploymentType(deploymentType)) + //ss = append(ss, schemas) + var f deployer.AnyConnectorFactory + switch v2[schemas.DiscriminatorFieldName()] { + case "docker": + f = deployer.Any(docker.NewFactory()) + case "podman": + f = deployer.Any(podman.NewFactory()) + case "kubernetes": + f = deployer.Any(kubernetes.NewFactory()) + case "python": + f = deployer.Any(python.NewFactory()) + } + + factories = append(factories, f) + //unserializedConfig, err := schemas.Unserialize(value) + + //if err != nil { + // return nil, err + //} + //reflectedConfig := reflect.ValueOf(unserializedConfig) + //fmt.Printf("%v\n", reflectedConfig) + + //reglist := DefaultDeployerRegistry.List() + //for _, factory := range reglist { + // if factory.ReflectedType() == reflectedConfig.Type() { + //dc := deployer.DeploymentConfig(unserializedConfig) + //unserializedConfig.(deployer.DeploymentConfig).NewFactory() + //t := reflect.TypeOf(unserializedConfig) + //v := reflect.ValueOf(unserializedConfig) + //fmt.Printf("%v\n", t) + //fmt.Printf("%v\n", v) + //d0 := reflect.ValueOf(unserializedConfig).Type() + //dc := deployer.DeploymentConfig[any]{} + ////dc := deployer.DeploymentConfig[reflect.TypeFor(unserializedConfig)]{} + //dc := deployer.DeploymentConfig[cfg](unserializedConfig) + //factories = append(factories, unserializedConfig) + //fmt.Printf("%v\n", unserializedConfig) + //factories = append(factories, + // deployer.Any( + // dc.NewFactory(), + // ), + //) + //} + //} + //} + } + + return registry.New(factories...), nil +} From aeb097b45877446f8b45a7f27f9abeb2f6941887 Mon Sep 17 00:00:00 2001 From: mleader Date: Mon, 30 Oct 2023 10:26:07 -0400 Subject: [PATCH 35/54] type switch on unserialized workflow deployer config --- new.go | 32 +++++++++++++++++++++++++------- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/new.go b/new.go index de81992f..e2985db1 100644 --- a/new.go +++ b/new.go @@ -51,8 +51,6 @@ func New( // deployer.Any(python.NewFactory()), //) -type cfg struct{} - func BuildRegistry(config map[string]any) (registry.Registry, error) { if config == nil { return nil, fmt.Errorf("the deployer configuration cannot be nil") @@ -63,6 +61,7 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { //} factories := make([]deployer.AnyConnectorFactory, 0) + workshops := make([]deployer.AnyConnectorFactory, 0) for deploymentType, value := range config { v2 := make(map[string]any) for k, v := range value.(map[any]any) { @@ -83,11 +82,30 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { } factories = append(factories, f) - //unserializedConfig, err := schemas.Unserialize(value) + unserializedConfig, err := schemas.Unserialize(value) + if err != nil { + return nil, err + } + fmt.Printf("%T\n", unserializedConfig) + //c2 := deployer.AnyDeploymentConfig(unserializedConfig) + //fmt.Printf("%v\n", c2) + //f2 := deployer.DeploymentConfig[unserializedConfig.(type)](unserializedConfig).NewFactory() + var f2 deployer.AnyConnectorFactory + // + switch unserializedConfig.(type) { + case docker.Config: + f2 = deployer.Any(docker.NewFactory()) + case podman.Config: + f2 = deployer.Any(podman.NewFactory()) + case kubernetes.Config: + f2 = deployer.Any(kubernetes.NewFactory()) + case python.Config: + f2 = deployer.Any(python.NewFactory()) + } + + workshops = append(workshops, f2) + //anydc := deployer.AnyDConfig(unserializedConfig).NewFactory() - //if err != nil { - // return nil, err - //} //reflectedConfig := reflect.ValueOf(unserializedConfig) //fmt.Printf("%v\n", reflectedConfig) @@ -116,5 +134,5 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { //} } - return registry.New(factories...), nil + return registry.New(workshops...), nil } From 880866f305af4d9861a95fa67000df5ff507382b Mon Sep 17 00:00:00 2001 From: mleader Date: Mon, 30 Oct 2023 10:28:45 -0400 Subject: [PATCH 36/54] remove dead code --- new.go | 77 +++++++++++++++++----------------------------------------- 1 file changed, 22 insertions(+), 55 deletions(-) diff --git a/new.go b/new.go index e2985db1..837d19b8 100644 --- a/new.go +++ b/new.go @@ -55,43 +55,38 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { if config == nil { return nil, fmt.Errorf("the deployer configuration cannot be nil") } + + //factories := make([]deployer.AnyConnectorFactory, 0) - //for _,v := range DefaultDeployerRegistry { - //fmt.Printf("%v\n" v) - //} - - factories := make([]deployer.AnyConnectorFactory, 0) workshops := make([]deployer.AnyConnectorFactory, 0) + for deploymentType, value := range config { - v2 := make(map[string]any) - for k, v := range value.(map[any]any) { - v2[k.(string)] = v - } schemas := DefaultDeployerRegistry.DeployConfigSchema(deployer.DeploymentType(deploymentType)) - //ss = append(ss, schemas) - var f deployer.AnyConnectorFactory - switch v2[schemas.DiscriminatorFieldName()] { - case "docker": - f = deployer.Any(docker.NewFactory()) - case "podman": - f = deployer.Any(podman.NewFactory()) - case "kubernetes": - f = deployer.Any(kubernetes.NewFactory()) - case "python": - f = deployer.Any(python.NewFactory()) - } + //v2 := make(map[string]any) + //for k, v := range value.(map[any]any) { + // v2[k.(string)] = v + //} + + //var f deployer.AnyConnectorFactory + //switch v2[schemas.DiscriminatorFieldName()] { + //case "docker": + // f = deployer.Any(docker.NewFactory()) + //case "podman": + // f = deployer.Any(podman.NewFactory()) + //case "kubernetes": + // f = deployer.Any(kubernetes.NewFactory()) + //case "python": + // f = deployer.Any(python.NewFactory()) + //} + // + //factories = append(factories, f) - factories = append(factories, f) unserializedConfig, err := schemas.Unserialize(value) if err != nil { return nil, err } - fmt.Printf("%T\n", unserializedConfig) - //c2 := deployer.AnyDeploymentConfig(unserializedConfig) - //fmt.Printf("%v\n", c2) - //f2 := deployer.DeploymentConfig[unserializedConfig.(type)](unserializedConfig).NewFactory() + var f2 deployer.AnyConnectorFactory - // switch unserializedConfig.(type) { case docker.Config: f2 = deployer.Any(docker.NewFactory()) @@ -104,34 +99,6 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { } workshops = append(workshops, f2) - //anydc := deployer.AnyDConfig(unserializedConfig).NewFactory() - - //reflectedConfig := reflect.ValueOf(unserializedConfig) - //fmt.Printf("%v\n", reflectedConfig) - - //reglist := DefaultDeployerRegistry.List() - //for _, factory := range reglist { - // if factory.ReflectedType() == reflectedConfig.Type() { - //dc := deployer.DeploymentConfig(unserializedConfig) - //unserializedConfig.(deployer.DeploymentConfig).NewFactory() - //t := reflect.TypeOf(unserializedConfig) - //v := reflect.ValueOf(unserializedConfig) - //fmt.Printf("%v\n", t) - //fmt.Printf("%v\n", v) - //d0 := reflect.ValueOf(unserializedConfig).Type() - //dc := deployer.DeploymentConfig[any]{} - ////dc := deployer.DeploymentConfig[reflect.TypeFor(unserializedConfig)]{} - //dc := deployer.DeploymentConfig[cfg](unserializedConfig) - //factories = append(factories, unserializedConfig) - //fmt.Printf("%v\n", unserializedConfig) - //factories = append(factories, - // deployer.Any( - // dc.NewFactory(), - // ), - //) - //} - //} - //} } return registry.New(workshops...), nil From f9791697a486d2eb97716fffa5b50cd3ccca3af5 Mon Sep 17 00:00:00 2001 From: mleader Date: Mon, 30 Oct 2023 11:09:37 -0400 Subject: [PATCH 37/54] use pointers --- config/load_test.go | 12 ++++-------- new.go | 34 ++++++++++++++++++++-------------- 2 files changed, 24 insertions(+), 22 deletions(-) diff --git a/config/load_test.go b/config/load_test.go index 660dc27b..dc2d613a 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -19,8 +19,7 @@ var configLoadData = map[string]struct { expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "docker"}, - "python": map[string]string{"deployer_id": "python"}, + "image": map[string]string{"deployer_id": "docker"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -36,8 +35,7 @@ log: expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "docker"}, - "python": map[string]string{"deployer_id": "python"}, + "image": map[string]string{"deployer_id": "docker"}, }, Log: log.Config{ Level: log.LevelDebug, @@ -54,8 +52,7 @@ deployers: expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "kubernetes"}, - "python": map[string]string{"deployer_id": "python"}, + "image": map[string]string{"deployer_id": "kubernetes"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -73,8 +70,7 @@ plugins: "quay.io/arcalot/example-plugin:latest", }, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "docker"}, - "python": map[string]string{"deployer_id": "python"}, + "image": map[string]string{"deployer_id": "docker"}, }, Log: log.Config{ Level: log.LevelInfo, diff --git a/new.go b/new.go index 837d19b8..7893be76 100644 --- a/new.go +++ b/new.go @@ -55,10 +55,10 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { if config == nil { return nil, fmt.Errorf("the deployer configuration cannot be nil") } - - //factories := make([]deployer.AnyConnectorFactory, 0) - workshops := make([]deployer.AnyConnectorFactory, 0) + factories := make([]deployer.AnyConnectorFactory, 0) + + //workshops := make([]deployer.AnyConnectorFactory, 0) for deploymentType, value := range config { schemas := DefaultDeployerRegistry.DeployConfigSchema(deployer.DeploymentType(deploymentType)) @@ -86,20 +86,26 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { return nil, err } - var f2 deployer.AnyConnectorFactory + //fmt.Printf("%v\n", unserializedConfig.(*docker.Config)) + //var f2 deployer.AnyConnectorFactory switch unserializedConfig.(type) { - case docker.Config: - f2 = deployer.Any(docker.NewFactory()) - case podman.Config: - f2 = deployer.Any(podman.NewFactory()) - case kubernetes.Config: - f2 = deployer.Any(kubernetes.NewFactory()) - case python.Config: - f2 = deployer.Any(python.NewFactory()) + case *docker.Config: + //f2 = deployer.Any(docker.NewFactory()) + factories = append(factories, deployer.Any(docker.NewFactory())) + case *podman.Config: + //f2 = deployer.Any(podman.NewFactory()) + factories = append(factories, deployer.Any(podman.NewFactory())) + case *kubernetes.Config: + //f2 = deployer.Any(kubernetes.NewFactory()) + factories = append(factories, deployer.Any(kubernetes.NewFactory())) + case *python.Config: + //f2 = deployer.Any(python.NewFactory()) + factories = append(factories, deployer.Any(python.NewFactory())) } - workshops = append(workshops, f2) + //workshops = append(workshops, f2) + //factories = append(factories, f2) } - return registry.New(workshops...), nil + return registry.New(factories...), nil } From c164285e06cc922595244865a27e6c6ebd6a68f8 Mon Sep 17 00:00:00 2001 From: mleader Date: Mon, 30 Oct 2023 11:37:12 -0400 Subject: [PATCH 38/54] default to builtin deployer stub --- new.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/new.go b/new.go index 7893be76..e8b13ff3 100644 --- a/new.go +++ b/new.go @@ -10,6 +10,7 @@ import ( kubernetes "go.flow.arcalot.io/kubernetesdeployer" podman "go.flow.arcalot.io/podmandeployer" python "go.flow.arcalot.io/pythondeployer" + testimpl "go.flow.arcalot.io/testdeployer" ) // New creates a new workflow engine with the provided configuration. The passed deployerRegistry is responsible for @@ -101,6 +102,8 @@ func BuildRegistry(config map[string]any) (registry.Registry, error) { case *python.Config: //f2 = deployer.Any(python.NewFactory()) factories = append(factories, deployer.Any(python.NewFactory())) + default: // deployer stub config + factories = append(factories, deployer.Any(testimpl.NewFactory())) } //workshops = append(workshops, f2) From d4f42ee5d23d115a8412e9d759b19f3906835f5f Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Mon, 30 Oct 2023 14:58:12 -0400 Subject: [PATCH 39/54] updated plugin provider schema --- internal/step/plugin/provider.go | 74 ++++++++++++++++++++------------ new.go | 7 --- workflow/executor_test.go | 6 ++- workflow/workflow_test.go | 8 ++-- 4 files changed, 55 insertions(+), 40 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 98f4a903..83f5c3b5 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -94,22 +94,42 @@ type Plugin struct { func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { return map[string]*schema.PropertySchema{ - "src": schema.NewPropertySchema( - schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), - schema.NewDisplayValue( - schema.PointerTo("Source"), - schema.PointerTo("Source file to be executed."), nil), - true, - nil, - nil, - nil, - nil, - []string{"\"quay.io/arcaflow/example-plugin:latest\""}, - ), - "deployment_type": schema.NewPropertySchema( - schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), + "plugin": schema.NewPropertySchema( + schema.NewObjectSchema( + "plugin_fields", + map[string]*schema.PropertySchema{ + "src": schema.NewPropertySchema( + schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), + schema.NewDisplayValue( + schema.PointerTo("Source"), + schema.PointerTo("Source file to be executed."), nil), + true, + nil, + nil, + nil, + nil, + []string{"\"quay.io/arcaflow/example-plugin:latest\""}, + ), + "deployment_type": schema.NewPropertySchema( + schema.NewStringSchema(schema.PointerTo[int64](1), nil, nil), + schema.NewDisplayValue( + schema.PointerTo("Type"), + schema.PointerTo( + fmt.Sprintf("Deployment type %s", + fmt.Sprintf(KeysString(DeploymentTypes)))), + nil, + ), + true, + nil, + nil, + nil, + nil, + []string{"docker"}, + ), + }, + ), schema.NewDisplayValue( - schema.PointerTo("Type"), + schema.PointerTo("Plugin Info"), schema.PointerTo( fmt.Sprintf("Deployment type %s", fmt.Sprintf(KeysString(DeploymentTypes)))), @@ -120,7 +140,7 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { nil, nil, nil, - []string{"docker"}, + nil, ), } } @@ -239,36 +259,36 @@ func (p *pluginProvider) Lifecycle() step.Lifecycle[step.LifecycleStage] { // LoadSchema deploys the plugin, connects to the plugin's ATP server, loads its schema, then // returns a runnableStep struct. Not to be confused with the runningStep struct. func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) (step.RunnableStep, error) { - pluginSrcInput := inputs["plugin"].(map[string]string) - requestedDeploymentType := deployer.DeploymentType(pluginSrcInput["deployment_type"]) - pluginSource := pluginSrcInput["src"] + pluginSrcInput := inputs["plugin"].(map[string]any) + requestedDeploymentType := deployer.DeploymentType(pluginSrcInput["deployment_type"].(string)) + pluginSource := pluginSrcInput["src"].(string) ctx, cancel := context.WithCancel(context.Background()) defer cancel() applicableLocalDeployer := p.localDeployers[requestedDeploymentType] - plugin, err := applicableLocalDeployer.Deploy(ctx, pluginSource) + plugin_connector, err := applicableLocalDeployer.Deploy(ctx, pluginSource) if err != nil { cancel() - return nil, fmt.Errorf("failed to deploy plugin of deployment type '%s' with source '%s' (%w)", + return nil, fmt.Errorf("failed to deploy plugin_connector of deployment type '%s' with source '%s' (%w)", requestedDeploymentType, pluginSource, err) } // Set up the ATP connection - transport := atp.NewClientWithLogger(plugin, p.logger) + transport := atp.NewClientWithLogger(plugin_connector, p.logger) // Read the schema information s, err := transport.ReadSchema() if err != nil { cancel() - return nil, fmt.Errorf("failed to read plugin schema from '%s' (%w)", pluginSource, err) + return nil, fmt.Errorf("failed to read plugin_connector schema from '%s' (%w)", pluginSource, err) } // Tell the server that the client is done if err := transport.Close(); err != nil { - return nil, fmt.Errorf("failed to instruct client to shut down plugin from source '%s' (%w)", pluginSource, err) + return nil, fmt.Errorf("failed to instruct client to shut down plugin_connector from source '%s' (%w)", pluginSource, err) } - // Shut down the plugin. - if err := plugin.Close(); err != nil { - return nil, fmt.Errorf("failed to shut down local plugin from '%s' (%w)", pluginSource, err) + // Shut down the plugin_connector. + if err := plugin_connector.Close(); err != nil { + return nil, fmt.Errorf("failed to shut down local plugin_connector from '%s' (%w)", pluginSource, err) } return &runnableStep{ diff --git a/new.go b/new.go index e8b13ff3..7caf9554 100644 --- a/new.go +++ b/new.go @@ -45,13 +45,6 @@ func New( }, nil } -//var DefaultDeployerRegistry = registry.New( -// deployer.Any(docker.NewFactory()), -// deployer.Any(kubernetes.NewFactory()), -// deployer.Any(podman.NewFactory()), -// deployer.Any(python.NewFactory()), -//) - func BuildRegistry(config map[string]any) (registry.Registry, error) { if config == nil { return nil, fmt.Errorf("the deployer configuration cannot be nil") diff --git a/workflow/executor_test.go b/workflow/executor_test.go index 67e23752..983fd085 100644 --- a/workflow/executor_test.go +++ b/workflow/executor_test.go @@ -64,8 +64,10 @@ func NewTestImplStepRegistry( pluginProvider := assert.NoErrorR[step.Provider](t)( plugin.New(logger, deployerRegistry, map[string]interface{}{ - "deployer_id": "test-impl", - "deploy_time": "0", + "builtin": map[string]any{ + "deployer_id": "test-impl", + "deploy_time": "0", + }, }), ) return assert.NoErrorR[step.Registry](t)(stepregistry.New( diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index 0e060dba..08961c0f 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -107,7 +107,7 @@ steps: end_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 80 @@ -115,14 +115,14 @@ steps: delay: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 50 last_step: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 0 @@ -133,7 +133,7 @@ steps: short_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: # End the test quickly. From 70942dfd1b767d40da45696ea1431e538863add5 Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Mon, 30 Oct 2023 16:32:35 -0400 Subject: [PATCH 40/54] fix some broken tests --- engine_test.go | 8 ++++-- internal/step/plugin/provider_test.go | 35 +++++++++++++++------------ workflow/workflow_test.go | 30 +++++++++++------------ 3 files changed, 41 insertions(+), 32 deletions(-) diff --git a/engine_test.go b/engine_test.go index 5496171f..f697c2e7 100644 --- a/engine_test.go +++ b/engine_test.go @@ -145,7 +145,9 @@ input: type_id: string steps: example: - plugin: quay.io/arcalot/arcaflow-plugin-template-python:0.2.1 + plugin: + src: quay.io/arcalot/arcaflow-plugin-template-python:0.2.1 + deployment_type: image input: name: !expr $.input.name output: @@ -176,7 +178,9 @@ input: type_id: string steps: example: - plugin: quay.io/arcalot/arcaflow-plugin-template-python:0.2.1 + plugin: + src: quay.io/arcalot/arcaflow-plugin-template-python:0.2.1 + deployment_type: image input: name: !expr $.input.name outputs: diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 12bd143c..b3035655 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -121,7 +121,7 @@ func TestProvider_MultipleDeployers(t *testing.T) { deployer.Any(testdeployer.NewFactory())) deployTimeMs := 20 workflowDeployerCfg := map[string]any{ - "image": map[string]any{ + "builtin": map[string]any{ "deployer_id": "test-impl", "deploy_time": deployTimeMs, "deploy_succeed": true, @@ -143,9 +143,9 @@ func TestProvider_MultipleDeployers(t *testing.T) { assert.NotNil(t, plp.Lifecycle()) stepSchema := map[string]any{ - "plugin": map[string]string{ - "src": "simulation", - "deployer_id": "image", + "plugin": map[string]any{ + "src": "simulation", + "deployment_type": "builtin", }, } byteSchema := map[string][]byte{} @@ -168,7 +168,7 @@ func TestProvider_MultipleDeployers(t *testing.T) { func TestProvider_Utility(t *testing.T) { workflowDeployerCfg := map[string]any{ - "image": map[string]any{"deployer_id": "test-impl"}, + "builtin": map[string]any{"deployer_id": "test-impl"}, } plp, err := plugin.New( @@ -189,9 +189,9 @@ func TestProvider_Utility(t *testing.T) { assert.NotNil(t, plp.Lifecycle()) stepSchema := map[string]any{ - "plugin": map[string]string{ - "src": "simulation", - "type": "image", + "plugin": map[string]any{ + "src": "simulation", + "deployment_type": "builtin", }, } byteSchema := map[string][]byte{} @@ -220,7 +220,7 @@ func TestProvider_HappyError(t *testing.T) { }, ) workflowDeployerCfg := map[string]any{ - "image": map[string]any{"deployer_id": "test-impl"}, + "builtin": map[string]any{"deployer_id": "test-impl"}, } deployerRegistry := deployer_registry.New( @@ -241,9 +241,9 @@ func TestProvider_HappyError(t *testing.T) { assert.NoError(t, err) stepSchema := map[string]any{ - "plugin": map[string]string{ + "plugin": map[string]any{ "src": "simulation", - "type": "image"}, + "type": "builtin"}, } byteSchema := map[string][]byte{} @@ -350,7 +350,12 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { assert.NoError(t, err) runnable, err := plp.LoadSchema( - map[string]any{"plugin": "simulation"}, map[string][]byte{}) + map[string]any{ + "plugin": map[string]any{ + "src": "simulation", + "deployment_type": "builtin", + }}, + map[string][]byte{}) assert.NoError(t, err) assert.NotNil(t, runnable) @@ -392,7 +397,7 @@ func TestProvider_DeployFail(t *testing.T) { deployer.Any(testdeployer.NewFactory())) deployTimeMs := 20 workflowDeployerCfg := map[string]any{ - "image": map[string]any{"deployer_id": "test-impl"}, + "builtin": map[string]any{"deployer_id": "test-impl"}, "deploy_time": deployTimeMs, "deploy_succeed": true, } @@ -405,7 +410,7 @@ func TestProvider_DeployFail(t *testing.T) { assert.NoError(t, err) stepSchema := map[string]any{ - "plugin": map[string]string{ + "plugin": map[string]any{ "src": "simulation", "type": "image"}, } @@ -468,7 +473,7 @@ func TestProvider_StartFail(t *testing.T) { // "deploy_succeed": true, //} workflowDeployerCfg := map[string]any{ - "image": map[string]any{"deployer_id": "test-impl"}, + "builtin": map[string]any{"deployer_id": "test-impl"}, //"deploy_time": deployTimeMs, //"deploy_succeed": true, } diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index 08961c0f..fcd8fa42 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -57,7 +57,7 @@ steps: long_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 2000 @@ -65,7 +65,7 @@ steps: short_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: # It needs to be long enough for it to ensure that long_wait is in a running state. @@ -181,14 +181,14 @@ steps: end_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 100 step_to_cancel: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 0 @@ -196,12 +196,12 @@ steps: stop_if: !expr $.steps.short_wait.outputs # Delay needs to be delayed long enough to ensure that it's in a deploy state when it's cancelled by short_wait deploy: - type: "test-impl" - deploy_time: 50 # 50 ms + deployer_id: "test-impl" + deploy_time: 50 # 50 ms short_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: # End the test quickly. @@ -247,7 +247,7 @@ steps: wait_1: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 0 @@ -278,7 +278,7 @@ steps: first_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: # Note: 5ms left only a 2.5ms margin for error. 10ms left almost 6ms. So 10ms min is recommended. @@ -286,7 +286,7 @@ steps: second_wait: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 10 @@ -358,18 +358,18 @@ steps: wait_1: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 0 deploy: - type: "test-impl" + deployer_id: "test-impl" #deploy_time: 20000 # 10 ms deploy_succeed: false wait_2: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait wait_for: !expr $.steps.wait_1.outputs.success input: @@ -401,7 +401,7 @@ steps: wait_1: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait input: wait_time_ms: 0 @@ -409,7 +409,7 @@ steps: plugin: src: "n/a" - type: "builtin" + deployment_type: "builtin" step: wait # No stop_if, so this shouldn't happen. wait_for: !expr $.steps.wait_1.outputs.cancelled_early From cc9f0d271127c5cd4c3e551ee335f5d03c95d1a7 Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Mon, 30 Oct 2023 16:46:59 -0400 Subject: [PATCH 41/54] fix more broken tests --- internal/step/plugin/provider_test.go | 125 ++++++++++++++------------ 1 file changed, 68 insertions(+), 57 deletions(-) diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index b3035655..c88e6dd5 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -220,7 +220,8 @@ func TestProvider_HappyError(t *testing.T) { }, ) workflowDeployerCfg := map[string]any{ - "builtin": map[string]any{"deployer_id": "test-impl"}, + "builtin": map[string]any{ + "deployer_id": "test-impl"}, } deployerRegistry := deployer_registry.New( @@ -229,7 +230,20 @@ func TestProvider_HappyError(t *testing.T) { _, err := plugin.New( logger, deployerRegistry, - map[string]any{"deployer_cfg": "bad"}, + map[string]any{ + "wrong": map[string]any{ + "deployer_id": "test-impl", + }}, + ) + assert.Error(t, err) + + _, err = plugin.New( + logger, + deployerRegistry, + map[string]any{ + "builtin": map[string]any{ + "deployer_id": "bad", + }}, ) assert.Error(t, err) @@ -242,8 +256,8 @@ func TestProvider_HappyError(t *testing.T) { stepSchema := map[string]any{ "plugin": map[string]any{ - "src": "simulation", - "type": "builtin"}, + "src": "simulation", + "deployment_type": "builtin"}, } byteSchema := map[string][]byte{} @@ -336,7 +350,9 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { }, ) workflowDeployerCfg := map[string]any{ - "deployer_id": "test-impl", + "builtin": map[string]any{ + "deployer_id": "test-impl", + }, } deployerRegistry := deployer_registry.New( @@ -411,52 +427,52 @@ func TestProvider_DeployFail(t *testing.T) { stepSchema := map[string]any{ "plugin": map[string]any{ - "src": "simulation", - "type": "image"}, + "src": "simulation", + "deployment_type": "builtin"}, } byteSchema := map[string][]byte{} - _, err = plp.LoadSchema(stepSchema, byteSchema) + runnable, err := plp.LoadSchema(stepSchema, byteSchema) assert.NoError(t, err) - //handler := &deployFailStageChangeHandler{ - // message: make(chan string), - //} - // - //// wait step - //running, err := runnable.Start(map[string]any{"step": "wait"}, t.Name(), handler) - //assert.NoError(t, err) - // - ////assert.NoError(t, running.ProvideStageInput( - //// string(plugin.StageIDDeploy), - //// map[string]any{ - //// "deploy": map[string]any{ - //// "type": "test-impl", - //// //"deploy_succeed": false, - //// //"deploy_time": deployTimeMs, - //// }, - //// }, - ////)) - //// - ////waitTimeMs := 50 - ////assert.NoError(t, running.ProvideStageInput( - //// string(plugin.StageIDStarting), - //// map[string]any{"input": map[string]any{"wait_time_ms": waitTimeMs}}, - ////)) - //// - ////message := <-handler.message - ////assert.Equals(t, - //// message, - //// fmt.Sprintf("intentional deployment fail after %d ms", deployTimeMs)) - //// - ////assert.Equals(t, string(running.State()), - //// string(step.RunningStepStateFinished)) - //// - ////assert.Equals(t, running.CurrentStage(), string(plugin.StageIDDeployFailed)) - //// - ////t.Cleanup(func() { - //// assert.NoError(t, running.Close()) - ////}) + handler := &deployFailStageChangeHandler{ + message: make(chan string), + } + + // wait step + running, err := runnable.Start(map[string]any{"step": "wait"}, t.Name(), handler) + assert.NoError(t, err) + + assert.NoError(t, running.ProvideStageInput( + string(plugin.StageIDDeploy), + map[string]any{ + "deploy": map[string]any{ + "deployer_id": "test-impl", + "deploy_succeed": false, + "deploy_time": deployTimeMs, + }, + }, + )) + + waitTimeMs := 50 + assert.NoError(t, running.ProvideStageInput( + string(plugin.StageIDStarting), + map[string]any{"input": map[string]any{"wait_time_ms": waitTimeMs}}, + )) + + message := <-handler.message + assert.Equals(t, + message, + fmt.Sprintf("intentional deployment fail after %d ms", deployTimeMs)) + + assert.Equals(t, string(running.State()), + string(step.RunningStepStateFinished)) + + assert.Equals(t, running.CurrentStage(), string(plugin.StageIDDeployFailed)) + + t.Cleanup(func() { + assert.NoError(t, running.Close()) + }) } func TestProvider_StartFail(t *testing.T) { @@ -467,15 +483,10 @@ func TestProvider_StartFail(t *testing.T) { }, ) deployTimeMs := 20 - //workflowDeployerCfg := map[string]any{ - // "type": "test-impl", - // "deploy_time": deployTimeMs, - // "deploy_succeed": true, - //} workflowDeployerCfg := map[string]any{ - "builtin": map[string]any{"deployer_id": "test-impl"}, - //"deploy_time": deployTimeMs, - //"deploy_succeed": true, + "builtin": map[string]any{"deployer_id": "test-impl"}, + "deploy_time": deployTimeMs, + "deploy_succeed": true, } plp, err := plugin.New( @@ -487,9 +498,9 @@ func TestProvider_StartFail(t *testing.T) { assert.NoError(t, err) stepSchema := map[string]any{ - "plugin": map[string]string{ - "src": "simulation", - "type": "image"}, + "plugin": map[string]any{ + "src": "simulation", + "deployment_type": "builtin"}, } byteSchema := map[string][]byte{} From de794a26a62baa6e5b170dfe1a9a0a596c7de077 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Tue, 31 Oct 2023 13:05:08 -0400 Subject: [PATCH 42/54] Call close to improve error reporting in Python deployer --- internal/step/plugin/provider.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 83f5c3b5..1d70f1da 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -271,7 +271,7 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) plugin_connector, err := applicableLocalDeployer.Deploy(ctx, pluginSource) if err != nil { cancel() - return nil, fmt.Errorf("failed to deploy plugin_connector of deployment type '%s' with source '%s' (%w)", + return nil, fmt.Errorf("failed to deploy plugin of deployment type '%s' with source '%s' (%w)", requestedDeploymentType, pluginSource, err) } // Set up the ATP connection @@ -280,15 +280,17 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) s, err := transport.ReadSchema() if err != nil { cancel() - return nil, fmt.Errorf("failed to read plugin_connector schema from '%s' (%w)", pluginSource, err) + // Close it. This allows it go get the error messages. + deployerErr := plugin_connector.Close() + return nil, fmt.Errorf("failed to read plugin schema from '%s' (%w). Deployer close error: %s", pluginSource, err, deployerErr) } // Tell the server that the client is done if err := transport.Close(); err != nil { - return nil, fmt.Errorf("failed to instruct client to shut down plugin_connector from source '%s' (%w)", pluginSource, err) + return nil, fmt.Errorf("failed to instruct client to shut down plugin from source '%s' (%w)", pluginSource, err) } - // Shut down the plugin_connector. + // Shut down the plugin. if err := plugin_connector.Close(); err != nil { - return nil, fmt.Errorf("failed to shut down local plugin_connector from '%s' (%w)", pluginSource, err) + return nil, fmt.Errorf("failed to shut down local plugin from '%s' (%w)", pluginSource, err) } return &runnableStep{ From b7944044815f066dc14fc81f55520adbd55c9aff Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Tue, 31 Oct 2023 14:40:32 -0400 Subject: [PATCH 43/54] remove dead code --- internal/step/plugin/provider.go | 50 +++++++------- internal/step/plugin/provider_test.go | 96 ++++++++++----------------- new.go | 88 +----------------------- steps.go | 7 +- 4 files changed, 63 insertions(+), 178 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 1d70f1da..6132b2f2 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -28,30 +28,26 @@ const errorStr = "error" // For more info, see `config/schema.go` func New(logger log.Logger, deployerRegistry registry.Registry, localDeployerConfigs map[string]any) (step.Provider, error) { localDeployers := make(map[deployer.DeploymentType]deployer.Connector) - for _, deploymentType := range deployerRegistry.DeploymentTypes() { - _, configHasType := localDeployerConfigs[string(deploymentType)] - if !configHasType { - logger.Infof("Config does not have config for deployment type '%s'. This type will not be usable.", deploymentType) - continue - } - localDeployerConfig, configPresent := localDeployerConfigs[string(deploymentType)] - var unserializedLocalDeployerConfig any - var err error - if configPresent { - unserializedLocalDeployerConfig, err = deployerRegistry.DeployConfigSchema(deploymentType).Unserialize(localDeployerConfig) - if err != nil { - return nil, fmt.Errorf("failed to load local deployer configuration, please check your Arcaflow configuration file (%w)", err) - } - } else { - logger.Warningf("No deployer configuration for type '%s' (%w). The workflow will fail if this type is needed", - deploymentType, err) + + // Build local deployers from requested deployers in engine workflow config. + for reqDeploymentType, deployerConfig := range localDeployerConfigs { + reqDeploymentTypeType := deployer.DeploymentType(reqDeploymentType) + // Unserialize config using deployer's schema in registry. + // This will return an error if the requested deployment type + // is not in the registry. + unserializedLocalDeployerConfig, err := deployerRegistry.DeployConfigSchema( + reqDeploymentTypeType).Unserialize(deployerConfig) + if err != nil { + return nil, fmt.Errorf("failed to load requested deployer type %s from workflow config (%w)", + reqDeploymentType, err) } - localDeployer, err := deployerRegistry.Create(deploymentType, unserializedLocalDeployerConfig, logger.WithLabel("source", "deployer")) + localDeployer, err := deployerRegistry.Create(reqDeploymentTypeType, + unserializedLocalDeployerConfig, logger.WithLabel("source", "deployer")) if err != nil { return nil, fmt.Errorf("invalid local deployer configuration, please check your Arcaflow configuration file (%w)", err) } - localDeployers[deploymentType] = localDeployer + localDeployers[reqDeploymentTypeType] = localDeployer } return &pluginProvider{ @@ -79,10 +75,10 @@ var DeploymentTypes = map[string]struct{}{ "python": struct{}{}, } -func KeysString(m map[string]struct{}) string { +func KeysString(m []deployer.DeploymentType) string { keys := make([]string, 0, len(m)) - for k := range m { - keys = append(keys, k) + for _, k := range m { + keys = append(keys, string(k)) } return "[" + strings.Join(keys, ", ") + "]" } @@ -115,8 +111,8 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { schema.NewDisplayValue( schema.PointerTo("Type"), schema.PointerTo( - fmt.Sprintf("Deployment type %s", - fmt.Sprintf(KeysString(DeploymentTypes)))), + fmt.Sprintf("Deployment type [%s]", + fmt.Sprintf(KeysString(p.deployerRegistry.DeploymentTypes()), ","))), nil, ), true, @@ -266,8 +262,10 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - applicableLocalDeployer := p.localDeployers[requestedDeploymentType] - + applicableLocalDeployer, ok := p.localDeployers[requestedDeploymentType] + if !ok { + return nil, fmt.Errorf("missing local deployer for requested type %s", requestedDeploymentType) + } plugin_connector, err := applicableLocalDeployer.Deploy(ctx, pluginSource) if err != nil { cancel() diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index c88e6dd5..bbff70d4 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -8,6 +8,7 @@ import ( deployer_registry "go.flow.arcalot.io/deployer/registry" "go.flow.arcalot.io/engine/internal/step" "go.flow.arcalot.io/engine/internal/step/plugin" + python "go.flow.arcalot.io/pythondeployer" testdeployer "go.flow.arcalot.io/testdeployer" "sync" "testing" @@ -118,7 +119,8 @@ func TestProvider_MultipleDeployers(t *testing.T) { }, ) deployerRegistry := deployer_registry.New( - deployer.Any(testdeployer.NewFactory())) + deployer.Any(testdeployer.NewFactory()), + deployer.Any(python.NewFactory())) deployTimeMs := 20 workflowDeployerCfg := map[string]any{ "builtin": map[string]any{ @@ -127,15 +129,11 @@ func TestProvider_MultipleDeployers(t *testing.T) { "deploy_succeed": true, }, "python": map[string]any{ - "deployer_id": "test-impl", + "deployer_id": "python", }, } - plp, err := plugin.New( - logger, - deployerRegistry, - workflowDeployerCfg, - ) + plp, err := plugin.New(logger, deployerRegistry, workflowDeployerCfg) assert.NoError(t, err) assert.Equals(t, plp.Kind(), "plugin") assert.NotNil(t, plp.ProviderSchema()) @@ -171,17 +169,13 @@ func TestProvider_Utility(t *testing.T) { "builtin": map[string]any{"deployer_id": "test-impl"}, } - plp, err := plugin.New( - log.New( - log.Config{ - Level: log.LevelError, - Destination: log.DestinationStdout, - }, - ), - deployer_registry.New( - deployer.Any(testdeployer.NewFactory())), - workflowDeployerCfg, - ) + plp, err := plugin.New(log.New( + log.Config{ + Level: log.LevelError, + Destination: log.DestinationStdout, + }, + ), deployer_registry.New( + deployer.Any(testdeployer.NewFactory())), workflowDeployerCfg) assert.NoError(t, err) assert.Equals(t, plp.Kind(), "plugin") assert.NotNil(t, plp.ProviderSchema()) @@ -227,31 +221,19 @@ func TestProvider_HappyError(t *testing.T) { deployerRegistry := deployer_registry.New( deployer.Any(testdeployer.NewFactory())) - _, err := plugin.New( - logger, - deployerRegistry, - map[string]any{ - "wrong": map[string]any{ - "deployer_id": "test-impl", - }}, - ) + _, err := plugin.New(logger, deployerRegistry, map[string]any{ + "wrong": map[string]any{ + "deployer_id": "test-impl", + }}) assert.Error(t, err) - _, err = plugin.New( - logger, - deployerRegistry, - map[string]any{ - "builtin": map[string]any{ - "deployer_id": "bad", - }}, - ) + _, err = plugin.New(logger, deployerRegistry, map[string]any{ + "builtin": map[string]any{ + "deployer_id": "bad", + }}) assert.Error(t, err) - plp, err := plugin.New( - logger, - deployerRegistry, - workflowDeployerCfg, - ) + plp, err := plugin.New(logger, deployerRegistry, workflowDeployerCfg) assert.NoError(t, err) stepSchema := map[string]any{ @@ -358,11 +340,7 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { deployerRegistry := deployer_registry.New( deployer.Any(testdeployer.NewFactory())) - plp, err := plugin.New( - logger, - deployerRegistry, - workflowDeployerCfg, - ) + plp, err := plugin.New(logger, deployerRegistry, workflowDeployerCfg) assert.NoError(t, err) runnable, err := plp.LoadSchema( @@ -413,16 +391,14 @@ func TestProvider_DeployFail(t *testing.T) { deployer.Any(testdeployer.NewFactory())) deployTimeMs := 20 workflowDeployerCfg := map[string]any{ - "builtin": map[string]any{"deployer_id": "test-impl"}, - "deploy_time": deployTimeMs, - "deploy_succeed": true, + "builtin": map[string]any{ + "deployer_id": "test-impl", + "deploy_time": deployTimeMs, + "deploy_succeed": true, + }, } - plp, err := plugin.New( - logger, - deployerRegistry, - workflowDeployerCfg, - ) + plp, err := plugin.New(logger, deployerRegistry, workflowDeployerCfg) assert.NoError(t, err) stepSchema := map[string]any{ @@ -484,17 +460,15 @@ func TestProvider_StartFail(t *testing.T) { ) deployTimeMs := 20 workflowDeployerCfg := map[string]any{ - "builtin": map[string]any{"deployer_id": "test-impl"}, - "deploy_time": deployTimeMs, - "deploy_succeed": true, + "builtin": map[string]any{ + "deployer_id": "test-impl", + "deploy_time": deployTimeMs, + "deploy_succeed": true, + }, } - plp, err := plugin.New( - logger, - deployer_registry.New( - deployer.Any(testdeployer.NewFactory())), - workflowDeployerCfg, - ) + plp, err := plugin.New(logger, deployer_registry.New( + deployer.Any(testdeployer.NewFactory())), workflowDeployerCfg) assert.NoError(t, err) stepSchema := map[string]any{ diff --git a/new.go b/new.go index 7caf9554..c923f7a3 100644 --- a/new.go +++ b/new.go @@ -1,16 +1,8 @@ package engine import ( - "fmt" log "go.arcalot.io/log/v2" - "go.flow.arcalot.io/deployer" - "go.flow.arcalot.io/deployer/registry" - docker "go.flow.arcalot.io/dockerdeployer" "go.flow.arcalot.io/engine/config" - kubernetes "go.flow.arcalot.io/kubernetesdeployer" - podman "go.flow.arcalot.io/podmandeployer" - python "go.flow.arcalot.io/pythondeployer" - testimpl "go.flow.arcalot.io/testdeployer" ) // New creates a new workflow engine with the provided configuration. The passed deployerRegistry is responsible for @@ -19,22 +11,9 @@ func New( config *config.Config, ) (WorkflowEngine, error) { logger := log.New(config.Log) - // TODO: Create new registry function - // need to build/create new deployer registry based on - // deployer_ids in config.LocalDeployers - // use registry.Create and its reflection as inspo - //for key, value := range config.LocalDeployers { - // - //} - reg, err := BuildRegistry(config.LocalDeployers) - if err != nil { - return nil, err - } - stepRegistry, err := NewDefaultStepRegistry( - logger, - reg, - config, - ) + + stepRegistry, err := NewDefaultStepRegistry(logger, + DefaultDeployerRegistry, config) if err != nil { return nil, err } @@ -44,64 +23,3 @@ func New( stepRegistry: stepRegistry, }, nil } - -func BuildRegistry(config map[string]any) (registry.Registry, error) { - if config == nil { - return nil, fmt.Errorf("the deployer configuration cannot be nil") - } - - factories := make([]deployer.AnyConnectorFactory, 0) - - //workshops := make([]deployer.AnyConnectorFactory, 0) - - for deploymentType, value := range config { - schemas := DefaultDeployerRegistry.DeployConfigSchema(deployer.DeploymentType(deploymentType)) - //v2 := make(map[string]any) - //for k, v := range value.(map[any]any) { - // v2[k.(string)] = v - //} - - //var f deployer.AnyConnectorFactory - //switch v2[schemas.DiscriminatorFieldName()] { - //case "docker": - // f = deployer.Any(docker.NewFactory()) - //case "podman": - // f = deployer.Any(podman.NewFactory()) - //case "kubernetes": - // f = deployer.Any(kubernetes.NewFactory()) - //case "python": - // f = deployer.Any(python.NewFactory()) - //} - // - //factories = append(factories, f) - - unserializedConfig, err := schemas.Unserialize(value) - if err != nil { - return nil, err - } - - //fmt.Printf("%v\n", unserializedConfig.(*docker.Config)) - //var f2 deployer.AnyConnectorFactory - switch unserializedConfig.(type) { - case *docker.Config: - //f2 = deployer.Any(docker.NewFactory()) - factories = append(factories, deployer.Any(docker.NewFactory())) - case *podman.Config: - //f2 = deployer.Any(podman.NewFactory()) - factories = append(factories, deployer.Any(podman.NewFactory())) - case *kubernetes.Config: - //f2 = deployer.Any(kubernetes.NewFactory()) - factories = append(factories, deployer.Any(kubernetes.NewFactory())) - case *python.Config: - //f2 = deployer.Any(python.NewFactory()) - factories = append(factories, deployer.Any(python.NewFactory())) - default: // deployer stub config - factories = append(factories, deployer.Any(testimpl.NewFactory())) - } - - //workshops = append(workshops, f2) - //factories = append(factories, f2) - } - - return registry.New(factories...), nil -} diff --git a/steps.go b/steps.go index 6ed80f8f..9840f752 100644 --- a/steps.go +++ b/steps.go @@ -2,7 +2,6 @@ package engine import ( "fmt" - "go.arcalot.io/log/v2" deployerRegistry "go.flow.arcalot.io/deployer/registry" "go.flow.arcalot.io/engine/config" @@ -14,11 +13,7 @@ import ( ) // NewDefaultStepRegistry creates a registry with the default step types applied. -func NewDefaultStepRegistry( - logger log.Logger, - deployerRegistry deployerRegistry.Registry, - config *config.Config, -) (step.Registry, error) { +func NewDefaultStepRegistry(logger log.Logger, deployerRegistry deployerRegistry.Registry, config *config.Config) (step.Registry, error) { pluginProvider, err := plugin.New(logger, deployerRegistry, config.LocalDeployers) if err != nil { return nil, fmt.Errorf("failed to create plugin step provider (%w)", err) From 488489d7af63ffb5ea62f354c7ba583558e1aed0 Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Tue, 31 Oct 2023 14:45:56 -0400 Subject: [PATCH 44/54] fix lint and style --- internal/step/plugin/provider.go | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 6132b2f2..ef34f276 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -70,12 +70,7 @@ type pluginProvider struct { func (p *pluginProvider) Register(_ step.Registry) { } -var DeploymentTypes = map[string]struct{}{ - "image": struct{}{}, - "python": struct{}{}, -} - -func KeysString(m []deployer.DeploymentType) string { +func keysString(m []deployer.DeploymentType) string { keys := make([]string, 0, len(m)) for _, k := range m { keys = append(keys, string(k)) @@ -112,7 +107,7 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { schema.PointerTo("Type"), schema.PointerTo( fmt.Sprintf("Deployment type [%s]", - fmt.Sprintf(KeysString(p.deployerRegistry.DeploymentTypes()), ","))), + keysString(p.deployerRegistry.DeploymentTypes()))), nil, ), true, @@ -128,7 +123,7 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { schema.PointerTo("Plugin Info"), schema.PointerTo( fmt.Sprintf("Deployment type %s", - fmt.Sprintf(KeysString(DeploymentTypes)))), + keysString(p.deployerRegistry.DeploymentTypes()))), nil, ), true, @@ -266,20 +261,20 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) if !ok { return nil, fmt.Errorf("missing local deployer for requested type %s", requestedDeploymentType) } - plugin_connector, err := applicableLocalDeployer.Deploy(ctx, pluginSource) + pluginConnector, err := applicableLocalDeployer.Deploy(ctx, pluginSource) if err != nil { cancel() return nil, fmt.Errorf("failed to deploy plugin of deployment type '%s' with source '%s' (%w)", requestedDeploymentType, pluginSource, err) } // Set up the ATP connection - transport := atp.NewClientWithLogger(plugin_connector, p.logger) + transport := atp.NewClientWithLogger(pluginConnector, p.logger) // Read the schema information s, err := transport.ReadSchema() if err != nil { cancel() // Close it. This allows it go get the error messages. - deployerErr := plugin_connector.Close() + deployerErr := pluginConnector.Close() return nil, fmt.Errorf("failed to read plugin schema from '%s' (%w). Deployer close error: %s", pluginSource, err, deployerErr) } // Tell the server that the client is done @@ -287,7 +282,7 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) return nil, fmt.Errorf("failed to instruct client to shut down plugin from source '%s' (%w)", pluginSource, err) } // Shut down the plugin. - if err := plugin_connector.Close(); err != nil { + if err := pluginConnector.Close(); err != nil { return nil, fmt.Errorf("failed to shut down local plugin from '%s' (%w)", pluginSource, err) } From 32bbcff029e5214ad9a31e4ca0a8b7b46bf3c83d Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 10:52:22 -0400 Subject: [PATCH 45/54] refactor to use deployer_name as discriminator value for factories --- config/load_test.go | 10 +++--- config/schema.go | 2 +- internal/step/dummy/provider_test.go | 6 ++-- internal/step/foreach/provider.go | 2 +- internal/step/lifecycle.go | 2 +- internal/step/plugin/provider.go | 8 ++--- internal/step/plugin/provider_test.go | 52 +++++++++++++-------------- internal/yaml/parser.go | 2 +- workflow/executor.go | 2 +- workflow/executor_test.go | 4 +-- workflow/model.go | 8 ++--- workflow/workflow.go | 20 +++++------ workflow/workflow_test.go | 4 +-- 13 files changed, 61 insertions(+), 61 deletions(-) diff --git a/config/load_test.go b/config/load_test.go index dc2d613a..daabe002 100644 --- a/config/load_test.go +++ b/config/load_test.go @@ -19,7 +19,7 @@ var configLoadData = map[string]struct { expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "docker"}, + "image": map[string]string{"deployer_name": "docker"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -35,7 +35,7 @@ log: expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "docker"}, + "image": map[string]string{"deployer_name": "docker"}, }, Log: log.Config{ Level: log.LevelDebug, @@ -47,12 +47,12 @@ log: input: ` deployers: image: - deployer_id: kubernetes + deployer_name: kubernetes `, expectedOutput: &config.Config{ TypeHintPlugins: nil, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "kubernetes"}, + "image": map[string]string{"deployer_name": "kubernetes"}, }, Log: log.Config{ Level: log.LevelInfo, @@ -70,7 +70,7 @@ plugins: "quay.io/arcalot/example-plugin:latest", }, LocalDeployers: map[string]any{ - "image": map[string]string{"deployer_id": "docker"}, + "image": map[string]string{"deployer_name": "docker"}, }, Log: log.Config{ Level: log.LevelInfo, diff --git a/config/schema.go b/config/schema.go index 35dee1b4..9bd216e5 100644 --- a/config/schema.go +++ b/config/schema.go @@ -63,7 +63,7 @@ func getConfigSchema() *schema.TypedScopeSchema[*Config] { nil, nil, nil, - schema.PointerTo(`{"image": {"deployer_id": "docker"}}`), + schema.PointerTo(`{"image": {"deployer_name": "docker"}}`), nil, ), "logged_outputs": schema.NewPropertySchema( diff --git a/internal/step/dummy/provider_test.go b/internal/step/dummy/provider_test.go index 622896e4..93625ecc 100644 --- a/internal/step/dummy/provider_test.go +++ b/internal/step/dummy/provider_test.go @@ -29,13 +29,13 @@ func (s *stageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } if *previousStageOutputID != "success" { - panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } message := (*previousStageOutput).(map[string]any)["message"].(string) s.message <- message diff --git a/internal/step/foreach/provider.go b/internal/step/foreach/provider.go index 55d3b58b..a5e63c24 100644 --- a/internal/step/foreach/provider.go +++ b/internal/step/foreach/provider.go @@ -471,7 +471,7 @@ func (r *runningStep) run() { } r.logger.Debugf("Executing item %d...", i) - // Ignore the output ID here because it can only be "success" + // Ignore the output Name here because it can only be "success" _, outputData, err := r.workflow.Execute(r.ctx, input) r.lock.Lock() if err != nil { diff --git a/internal/step/lifecycle.go b/internal/step/lifecycle.go index 83495250..dfd58574 100644 --- a/internal/step/lifecycle.go +++ b/internal/step/lifecycle.go @@ -49,7 +49,7 @@ func (l Lifecycle[StageType]) DAG() (dgraph.DirectedGraph[StageType], error) { // lifecycleStage is a helper interface for being able to construct a DAG from a lifecycle. type lifecycleStage interface { - // Identifier returns the ID of the stage. + // Identifier returns the Name of the stage. Identifier() string // NextStageIDs returns the next stage identifiers. NextStageIDs() []string diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index ef34f276..004e6f77 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -364,7 +364,7 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st cancelSignal := stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] if cancelSignal == nil { // Not present - stopIfProperty.Disable(fmt.Sprintf("Cancel signal with ID '%s' is not present in plugin '%s', step '%s'. Signal handler IDs present: %v", + stopIfProperty.Disable(fmt.Sprintf("Cancel signal with Name '%s' is not present in plugin '%s', step '%s'. Signal handler IDs present: %v", plugin.CancellationSignalSchema.ID(), r.source, stepID, reflect.ValueOf(stepSchema.SignalHandlers()).MapKeys())) } else if err := plugin.CancellationSignalSchema.DataSchemaValue.ValidateCompatibility(cancelSignal.DataSchemaValue); err != nil { // Present but incompatible @@ -579,10 +579,10 @@ type runningStep struct { runInputAvailable bool logger log.Logger currentStage StageID - runID string // The ID associated with this execution (the workflow step ID) + runID string // The Name associated with this execution (the workflow step Name) deploymentType deployer.DeploymentType source string - pluginStepID string // The ID of the step in the plugin + pluginStepID string // The Name of the step in the plugin state step.RunningStepState useLocalDeployer bool localDeployer deployer.Connector @@ -799,7 +799,7 @@ func (r *runningStep) run() { r.container = container } r.lock.Unlock() - r.logger.Debugf("Successfully deployed container with ID '%s' for step %s/%s", container.ID(), r.runID, r.pluginStepID) + r.logger.Debugf("Successfully deployed container with Name '%s' for step %s/%s", container.ID(), r.runID, r.pluginStepID) if err := r.startStage(container); err != nil { r.startFailed(err) return diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index bbff70d4..c4276a09 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -33,13 +33,13 @@ func (s *deployFailStageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } if *previousStageOutputID != "error" { - panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } message := (*previousStageOutput).(plugin.DeployFailed).Error @@ -65,13 +65,13 @@ func (s *startFailStageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } if *previousStageOutputID != "error" { - panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } message := (*previousStageOutput).(plugin.Crashed).Output @@ -98,13 +98,13 @@ func (s *stageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } if *previousStageOutputID != "success" { - panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output ID")) + panic(fmt.Errorf("no previous stage output Name")) } message := (*previousStageOutput).(map[any]any)["message"].(string) @@ -124,12 +124,12 @@ func TestProvider_MultipleDeployers(t *testing.T) { deployTimeMs := 20 workflowDeployerCfg := map[string]any{ "builtin": map[string]any{ - "deployer_id": "test-impl", + "deployer_name": "test-impl", "deploy_time": deployTimeMs, "deploy_succeed": true, }, "python": map[string]any{ - "deployer_id": "python", + "deployer_name": "python", }, } @@ -166,7 +166,7 @@ func TestProvider_MultipleDeployers(t *testing.T) { func TestProvider_Utility(t *testing.T) { workflowDeployerCfg := map[string]any{ - "builtin": map[string]any{"deployer_id": "test-impl"}, + "builtin": map[string]any{"deployer_name": "test-impl"}, } plp, err := plugin.New(log.New( @@ -215,7 +215,7 @@ func TestProvider_HappyError(t *testing.T) { ) workflowDeployerCfg := map[string]any{ "builtin": map[string]any{ - "deployer_id": "test-impl"}, + "deployer_name": "test-impl"}, } deployerRegistry := deployer_registry.New( @@ -223,13 +223,13 @@ func TestProvider_HappyError(t *testing.T) { _, err := plugin.New(logger, deployerRegistry, map[string]any{ "wrong": map[string]any{ - "deployer_id": "test-impl", + "deployer_name": "test-impl", }}) assert.Error(t, err) _, err = plugin.New(logger, deployerRegistry, map[string]any{ "builtin": map[string]any{ - "deployer_id": "bad", + "deployer_name": "bad", }}) assert.Error(t, err) @@ -266,23 +266,23 @@ func TestProvider_HappyError(t *testing.T) { assert.Error(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "deployer_id": "test-impl", - "deploy_time": "abc"}}, + "deployer_name": "test-impl", + "deploy_time": "abc"}}, )) assert.NoError(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "deployer_id": "test-impl", - "deploy_time": 1}}, + "deployer_name": "test-impl", + "deploy_time": 1}}, )) // provide deploy input a 2nd time assert.Error(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "deployer_id": "test-impl", - "deploy_time": nil}}, + "deployer_name": "test-impl", + "deploy_time": nil}}, )) // unserialize nil input schema error @@ -333,7 +333,7 @@ func TestProvider_VerifyCancelSignal(t *testing.T) { ) workflowDeployerCfg := map[string]any{ "builtin": map[string]any{ - "deployer_id": "test-impl", + "deployer_name": "test-impl", }, } @@ -392,7 +392,7 @@ func TestProvider_DeployFail(t *testing.T) { deployTimeMs := 20 workflowDeployerCfg := map[string]any{ "builtin": map[string]any{ - "deployer_id": "test-impl", + "deployer_name": "test-impl", "deploy_time": deployTimeMs, "deploy_succeed": true, }, @@ -423,7 +423,7 @@ func TestProvider_DeployFail(t *testing.T) { string(plugin.StageIDDeploy), map[string]any{ "deploy": map[string]any{ - "deployer_id": "test-impl", + "deployer_name": "test-impl", "deploy_succeed": false, "deploy_time": deployTimeMs, }, @@ -461,7 +461,7 @@ func TestProvider_StartFail(t *testing.T) { deployTimeMs := 20 workflowDeployerCfg := map[string]any{ "builtin": map[string]any{ - "deployer_id": "test-impl", + "deployer_name": "test-impl", "deploy_time": deployTimeMs, "deploy_succeed": true, }, @@ -492,7 +492,7 @@ func TestProvider_StartFail(t *testing.T) { assert.NoError(t, running.ProvideStageInput( string(plugin.StageIDDeploy), map[string]any{"deploy": map[string]any{ - "deployer_id": "test-impl", + "deployer_name": "test-impl", "deploy_succeed": true, "deploy_time": deployTimeMs, "disable_plugin_writes": true}}, diff --git a/internal/yaml/parser.go b/internal/yaml/parser.go index 9984b525..c602ec1c 100644 --- a/internal/yaml/parser.go +++ b/internal/yaml/parser.go @@ -107,7 +107,7 @@ func (n node) Raw() any { } return result default: - panic(fmt.Errorf("bug: unexpected type ID: %s", n.typeID)) + panic(fmt.Errorf("bug: unexpected type Name: %s", n.typeID)) } } diff --git a/workflow/executor.go b/workflow/executor.go index f1a4add4..02a6462c 100644 --- a/workflow/executor.go +++ b/workflow/executor.go @@ -58,7 +58,7 @@ type ExecutableWorkflow interface { // Execute runs a workflow until it finishes or until the context expires with the specified input. The input // must only contain primitives (float, int, bool, string, map, slice) and may not contain structs and other - // elements. The output will consist of the output ID, the returned output data corresponding to the output IDs + // elements. The output will consist of the output Name, the returned output data corresponding to the output IDs // schema, or if an error happened, the error. Execute( ctx context.Context, diff --git a/workflow/executor_test.go b/workflow/executor_test.go index 983fd085..1b5cd112 100644 --- a/workflow/executor_test.go +++ b/workflow/executor_test.go @@ -65,8 +65,8 @@ func NewTestImplStepRegistry( pluginProvider := assert.NoErrorR[step.Provider](t)( plugin.New(logger, deployerRegistry, map[string]interface{}{ "builtin": map[string]any{ - "deployer_id": "test-impl", - "deploy_time": "0", + "deployer_name": "test-impl", + "deploy_time": "0", }, }), ) diff --git a/workflow/model.go b/workflow/model.go index 5ff2b02a..3efe3b22 100644 --- a/workflow/model.go +++ b/workflow/model.go @@ -21,7 +21,7 @@ type Workflow struct { // Steps contains the possible steps in this workflow. The data set must contain a valid step structure where the // inputs to stages may consist only of primitive types and expressions. Steps map[string]any `json:"steps"` - // Outputs lets you define one or more outputs. The outputs should be keyed by their output ID (e.g. "success") and + // Outputs lets you define one or more outputs. The outputs should be keyed by their output Name (e.g. "success") and // the value should be the data you wish to output. The data may contain expressions to construct the output. Outputs map[string]any `json:"outputs"` // OutputSchema is an optional override for the automatically inferred output schema from the Outputs data and @@ -182,7 +182,7 @@ type DAGItem struct { StepID string // StageID is the stage of the step provider this item refers to. StageID string - // OutputID is the ID of the output of the step stage. + // OutputID is the Name of the output of the step stage. OutputID string // OutputSchema contains the output-specific schema for this item. OutputSchema schema.StepOutput @@ -212,12 +212,12 @@ func (d DAGItem) String() string { } } -// GetStageNodeID returns the DAG node ID for a stage. +// GetStageNodeID returns the DAG node Name for a stage. func GetStageNodeID(stepID string, stageID string) string { return fmt.Sprintf("steps.%s.%s", stepID, stageID) } -// GetOutputNodeID returns the DAG node ID for a stage output. +// GetOutputNodeID returns the DAG node Name for a stage output. func GetOutputNodeID(stepID string, stageID string, outputID string) string { return fmt.Sprintf("steps.%s.%s.%s", stepID, stageID, outputID) } diff --git a/workflow/workflow.go b/workflow/workflow.go index 9ffab54d..4e962576 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -260,31 +260,31 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo } stageNode, err := l.dag.GetNodeByID(GetStageNodeID(stepID, *previousStage)) if err != nil { - l.logger.Errorf("Failed to get stage node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) - l.recentErrors <- fmt.Errorf("failed to get stage node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.logger.Errorf("Failed to get stage node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.recentErrors <- fmt.Errorf("failed to get stage node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) l.cancel() return } l.logger.Debugf("Removed node '%s' from the DAG", stageNode.ID()) if err := stageNode.Remove(); err != nil { - l.logger.Errorf("Failed to remove stage node ID %s (%w)", stageNode.ID(), err) - l.recentErrors <- fmt.Errorf("failed to remove stage node ID %s (%w)", stageNode.ID(), err) + l.logger.Errorf("Failed to remove stage node Name %s (%w)", stageNode.ID(), err) + l.recentErrors <- fmt.Errorf("failed to remove stage node Name %s (%w)", stageNode.ID(), err) l.cancel() return } if previousStageOutputID != nil { outputNode, err := l.dag.GetNodeByID(GetOutputNodeID(stepID, *previousStage, *previousStageOutputID)) if err != nil { - l.logger.Errorf("Failed to get output node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) - l.recentErrors <- fmt.Errorf("failed to get output node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.logger.Errorf("Failed to get output node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.recentErrors <- fmt.Errorf("failed to get output node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) l.cancel() return } // Removes the node from the DAG. This results in the nodes not having inbound connections, allowing them to be processed. l.logger.Debugf("Removed node '%s' from the DAG", outputNode.ID()) if err := outputNode.Remove(); err != nil { - l.logger.Errorf("Failed to remove output node ID %s (%w)", outputNode.ID(), err) - l.recentErrors <- fmt.Errorf("failed to remove output node ID %s (%w)", outputNode.ID(), err) + l.logger.Errorf("Failed to remove output node Name %s (%w)", outputNode.ID(), err) + l.recentErrors <- fmt.Errorf("failed to remove output node Name %s (%w)", outputNode.ID(), err) l.cancel() return } @@ -293,7 +293,7 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo if stepLogConfig != nil { l.logger.Writef( stepLogConfig.LogLevel, - "Output ID for step \"%s\" is \"%s\".\nOutput data: \"%s\"", + "Output Name for step \"%s\" is \"%s\".\nOutput data: \"%s\"", stepID, *previousStageOutputID, *previousStageOutput, @@ -362,7 +362,7 @@ func (l *loopState) notifySteps() { //nolint:gocognit // This check is here just to make sure it has the required fields set if node.Item().StepID == "" || node.Item().StageID == "" { // This shouldn't happen - panic("Step or stage ID missing") + panic("Step or stage Name missing") } stageInputData := untypedInputData.(map[any]any) diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index fcd8fa42..c94488f2 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -196,7 +196,7 @@ steps: stop_if: !expr $.steps.short_wait.outputs # Delay needs to be delayed long enough to ensure that it's in a deploy state when it's cancelled by short_wait deploy: - deployer_id: "test-impl" + deployer_name: "test-impl" deploy_time: 50 # 50 ms short_wait: plugin: @@ -363,7 +363,7 @@ steps: input: wait_time_ms: 0 deploy: - deployer_id: "test-impl" + deployer_name: "test-impl" #deploy_time: 20000 # 10 ms deploy_succeed: false wait_2: From f8482c216d894fbadcc3a1a1391dd0433f21470b Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 17:01:22 -0400 Subject: [PATCH 46/54] update deployer dependencies --- go.mod | 16 ++++++++-------- go.sum | 32 ++++++++++++++++---------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/go.mod b/go.mod index de8d6e7d..c9102a61 100644 --- a/go.mod +++ b/go.mod @@ -7,14 +7,14 @@ require ( go.arcalot.io/dgraph v1.1.0 go.arcalot.io/lang v1.0.0 go.arcalot.io/log/v2 v2.0.0 - go.flow.arcalot.io/deployer v0.3.0 - go.flow.arcalot.io/dockerdeployer v0.4.0 + go.flow.arcalot.io/deployer v0.4.0 + go.flow.arcalot.io/dockerdeployer v0.5.0 go.flow.arcalot.io/expressions v0.2.1 - go.flow.arcalot.io/kubernetesdeployer v0.7.0 + go.flow.arcalot.io/kubernetesdeployer v0.4.0 go.flow.arcalot.io/pluginsdk v0.5.0 - go.flow.arcalot.io/podmandeployer v0.5.0 - go.flow.arcalot.io/pythondeployer v0.3.0 - go.flow.arcalot.io/testdeployer v0.3.0 + go.flow.arcalot.io/podmandeployer v0.6.0 + go.flow.arcalot.io/pythondeployer v0.4.0 + go.flow.arcalot.io/testdeployer v0.4.0 gopkg.in/yaml.v3 v3.0.1 ) @@ -24,7 +24,7 @@ require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/distribution/reference v0.5.0 // indirect github.com/docker/distribution v2.8.3+incompatible // indirect - github.com/docker/docker v24.0.6+incompatible // indirect + github.com/docker/docker v24.0.7+incompatible // indirect github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/emicklei/go-restful/v3 v3.10.0 // indirect @@ -50,7 +50,7 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/rogpeppe/go-internal v1.11.0 // indirect github.com/x448/float16 v0.8.4 // indirect - go.flow.arcalot.io/testplugin v0.2.1 // indirect + go.flow.arcalot.io/testplugin v0.3.0 // indirect golang.org/x/mod v0.13.0 // indirect golang.org/x/net v0.15.0 // indirect golang.org/x/oauth2 v0.2.0 // indirect diff --git a/go.sum b/go.sum index fc01050e..cfcba5ee 100644 --- a/go.sum +++ b/go.sum @@ -23,8 +23,8 @@ github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= github.com/docker/distribution v2.8.3+incompatible h1:AtKxIZ36LoNK51+Z6RpzLpddBirtxJnzDrHLEKxTAYk= github.com/docker/distribution v2.8.3+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= -github.com/docker/docker v24.0.6+incompatible h1:hceabKCtUgDqPu+qm0NgsaXf28Ljf4/pWFL7xjWWDgE= -github.com/docker/docker v24.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v24.0.7+incompatible h1:Wo6l37AuwP3JaMnZa226lzVXGA3F9Ig1seQen0cKYlM= +github.com/docker/docker v24.0.7+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= @@ -160,24 +160,24 @@ go.arcalot.io/lang v1.0.0 h1:mgDaieT4wWdZTnR4V7+/pgYRmzfU7VZZgIzHccuxAbY= go.arcalot.io/lang v1.0.0/go.mod h1:ALqfYEhAzC2WoGLaycmJoNJd5NmkR7V1PSKp/c5D278= go.arcalot.io/log/v2 v2.0.0 h1:mbmsWDVBXZNWrDzUh5JLzeGCQ59kTuMFs+pyfJGc1hk= go.arcalot.io/log/v2 v2.0.0/go.mod h1:1V8jnFIIGwh2CtcGkHNOmy1nCo7LbazQNkUcnKYNMn4= -go.flow.arcalot.io/deployer v0.3.0 h1:LPikgRG5jGA76W8JthycvzfREL5Y0+++KAiQxSnKhdU= -go.flow.arcalot.io/deployer v0.3.0/go.mod h1:x6gsz/hANR8qN1nerpyY3vXpdaqofDH5Wlg+Nsqg/x0= -go.flow.arcalot.io/dockerdeployer v0.4.0 h1:t5b8o3xfKKb/WIX558486csjo4uMQmAXsikBLsKFEIg= -go.flow.arcalot.io/dockerdeployer v0.4.0/go.mod h1:UZSM6buJBRlgCURUE/BVkak8tfAXzj3oeQBSRZECbSc= +go.flow.arcalot.io/deployer v0.4.0 h1:5YveLCX+zc8Ra/aukHOwD5OrJD2W8WRzoruf3bpJfqY= +go.flow.arcalot.io/deployer v0.4.0/go.mod h1:x6gsz/hANR8qN1nerpyY3vXpdaqofDH5Wlg+Nsqg/x0= +go.flow.arcalot.io/dockerdeployer v0.5.0 h1:CKsy5K38/0hF6dEry7f0YIfI5uNKGImyCwi8miwPEhE= +go.flow.arcalot.io/dockerdeployer v0.5.0/go.mod h1:zE/okmoc47PTdJxiag+Gk9LMzKR1GemeP1Q3woOY3/A= go.flow.arcalot.io/expressions v0.2.1 h1:TAAbDrgJJLpmgA5ASyP/KzrXWtpEaQ8JsCPHgpe5kLw= go.flow.arcalot.io/expressions v0.2.1/go.mod h1:Vw1ScNu4Uyw1/l87LAH8jxe0DyRWwMh+rlfB/BPYDOU= -go.flow.arcalot.io/kubernetesdeployer v0.7.0 h1:r41qWc/XiPy9l3cfMXZG8F2kGenRh1xsx2auim/Ydyw= -go.flow.arcalot.io/kubernetesdeployer v0.7.0/go.mod h1:VvU6duoo5NR2ITUhx/UCGrkdJnXIeYm+/yHmGKtkXsk= +go.flow.arcalot.io/kubernetesdeployer v0.4.0 h1:y5GaGIsTnnXroqnj8yczM82plQGNttMHVc4N41TCzxQ= +go.flow.arcalot.io/kubernetesdeployer v0.4.0/go.mod h1:XNkG7ARtjdHFigJLLIJ9kcofkq2DzZ8gz2FPCnJON+I= go.flow.arcalot.io/pluginsdk v0.5.0 h1:TRS/waCTcdoMZ9neDAcfy3zpzyDnPHRbhV+Y1kpcw3Y= go.flow.arcalot.io/pluginsdk v0.5.0/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= -go.flow.arcalot.io/podmandeployer v0.5.0 h1:h7hEhWUgxJzNKlEohZ+meKhl3FWjaXQahQ8vN3YVRNs= -go.flow.arcalot.io/podmandeployer v0.5.0/go.mod h1:36JCcTB6nauahcXUPfIpdEw7Zfp0ufM07o3VNTvrCc0= -go.flow.arcalot.io/pythondeployer v0.3.0 h1:ercLuDwFoDSL0f6YvZEqFW0/nO7Yv7DkbROl3rKxYDk= -go.flow.arcalot.io/pythondeployer v0.3.0/go.mod h1:ND1x/Vhu/6q50zQeisCcD6oQ6lKVJFflOrfDccnIjSY= -go.flow.arcalot.io/testdeployer v0.3.0 h1:Soyz+rDa3Y3VjWBGuL3zNlX3LM4uKp9Ex7///fCgrZA= -go.flow.arcalot.io/testdeployer v0.3.0/go.mod h1:Eel0ORhtKdYYDsd+e+btBBygIn+9Sz/b+JFDwH39VWI= -go.flow.arcalot.io/testplugin v0.2.1 h1:9kQ2MKvcXtEcwk5c4qSWN+FovpER2C9vn730laAm9iE= -go.flow.arcalot.io/testplugin v0.2.1/go.mod h1:ZoVF8tIKppQmj5nvoZPA48GQ7BuoWXQcuCw2x2sJxjE= +go.flow.arcalot.io/podmandeployer v0.6.0 h1:SlcQUU6xt24Oa0OFNnwYE+d+XQbDXErMLvpCi2gMHoA= +go.flow.arcalot.io/podmandeployer v0.6.0/go.mod h1:4wfcl0qjV02y64We3ZSDz+3lwdOfbe+gpFjm7SQKTRA= +go.flow.arcalot.io/pythondeployer v0.4.0 h1:l8nw6awYMVzgND+ZXdbnNJPYu3V0sgSUFsIzn+SRgh0= +go.flow.arcalot.io/pythondeployer v0.4.0/go.mod h1:me9SIMVTCBzCmceILdBMxXYrZGakiPOMasHgujmgJlE= +go.flow.arcalot.io/testdeployer v0.4.0 h1:helexgZOnYlbzU+egkxsLs95iMwE3lc+vp+Qbv2xwFI= +go.flow.arcalot.io/testdeployer v0.4.0/go.mod h1:Luw76oKeri40a7k8buk7Q7J86Tpt8lf9LTVZr6lqgTk= +go.flow.arcalot.io/testplugin v0.3.0 h1:LY0VWL1M0X+mSE0nVCwoD0PjQHqKhbka2Gc70uuP04k= +go.flow.arcalot.io/testplugin v0.3.0/go.mod h1:TW6h/kZyM+4gBBKqvoDI0XWXgNcUMKbDHaQOaKIRZdk= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= From a9e436824066bc6bedf31a398eb2de00a2a6abdd Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 17:16:47 -0400 Subject: [PATCH 47/54] fix error joining --- internal/step/plugin/provider.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 004e6f77..4833bd38 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -78,11 +78,6 @@ func keysString(m []deployer.DeploymentType) string { return "[" + strings.Join(keys, ", ") + "]" } -type Plugin struct { - Src string - Type string -} - func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { return map[string]*schema.PropertySchema{ "plugin": schema.NewPropertySchema( @@ -275,7 +270,8 @@ func (p *pluginProvider) LoadSchema(inputs map[string]any, _ map[string][]byte) cancel() // Close it. This allows it go get the error messages. deployerErr := pluginConnector.Close() - return nil, fmt.Errorf("failed to read plugin schema from '%s' (%w). Deployer close error: %s", pluginSource, err, deployerErr) + return nil, fmt.Errorf("failed to read plugin schema from '%s' (%w). Deployer close error: (%s)", + pluginSource, err, deployerErr.Error()) } // Tell the server that the client is done if err := transport.Close(); err != nil { From 7cb26db8ed7003b543f7edf2bcd0f83358e37fa9 Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 17:28:43 -0400 Subject: [PATCH 48/54] unfactor Name back to ID --- internal/step/dummy/provider_test.go | 6 +++--- internal/step/foreach/provider.go | 2 +- internal/step/lifecycle.go | 2 +- internal/step/plugin/provider.go | 2 +- internal/step/plugin/provider_test.go | 16 ++++++++-------- workflow/executor.go | 2 +- workflow/model.go | 8 ++++---- workflow/workflow.go | 20 ++++++++++---------- 8 files changed, 29 insertions(+), 29 deletions(-) diff --git a/internal/step/dummy/provider_test.go b/internal/step/dummy/provider_test.go index 93625ecc..622896e4 100644 --- a/internal/step/dummy/provider_test.go +++ b/internal/step/dummy/provider_test.go @@ -29,13 +29,13 @@ func (s *stageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } if *previousStageOutputID != "success" { - panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } message := (*previousStageOutput).(map[string]any)["message"].(string) s.message <- message diff --git a/internal/step/foreach/provider.go b/internal/step/foreach/provider.go index a5e63c24..55d3b58b 100644 --- a/internal/step/foreach/provider.go +++ b/internal/step/foreach/provider.go @@ -471,7 +471,7 @@ func (r *runningStep) run() { } r.logger.Debugf("Executing item %d...", i) - // Ignore the output Name here because it can only be "success" + // Ignore the output ID here because it can only be "success" _, outputData, err := r.workflow.Execute(r.ctx, input) r.lock.Lock() if err != nil { diff --git a/internal/step/lifecycle.go b/internal/step/lifecycle.go index dfd58574..83495250 100644 --- a/internal/step/lifecycle.go +++ b/internal/step/lifecycle.go @@ -49,7 +49,7 @@ func (l Lifecycle[StageType]) DAG() (dgraph.DirectedGraph[StageType], error) { // lifecycleStage is a helper interface for being able to construct a DAG from a lifecycle. type lifecycleStage interface { - // Identifier returns the Name of the stage. + // Identifier returns the ID of the stage. Identifier() string // NextStageIDs returns the next stage identifiers. NextStageIDs() []string diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index 4833bd38..bff40e82 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -360,7 +360,7 @@ func (r *runnableStep) Lifecycle(input map[string]any) (result step.Lifecycle[st cancelSignal := stepSchema.SignalHandlers()[plugin.CancellationSignalSchema.ID()] if cancelSignal == nil { // Not present - stopIfProperty.Disable(fmt.Sprintf("Cancel signal with Name '%s' is not present in plugin '%s', step '%s'. Signal handler IDs present: %v", + stopIfProperty.Disable(fmt.Sprintf("Cancel signal with ID '%s' is not present in plugin '%s', step '%s'. Signal handler IDs present: %v", plugin.CancellationSignalSchema.ID(), r.source, stepID, reflect.ValueOf(stepSchema.SignalHandlers()).MapKeys())) } else if err := plugin.CancellationSignalSchema.DataSchemaValue.ValidateCompatibility(cancelSignal.DataSchemaValue); err != nil { // Present but incompatible diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index c4276a09..6d749224 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -33,13 +33,13 @@ func (s *deployFailStageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } if *previousStageOutputID != "error" { - panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } message := (*previousStageOutput).(plugin.DeployFailed).Error @@ -65,13 +65,13 @@ func (s *startFailStageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } if *previousStageOutputID != "error" { panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } message := (*previousStageOutput).(plugin.Crashed).Output @@ -98,13 +98,13 @@ func (s *stageChangeHandler) OnStepComplete( panic(fmt.Errorf("invalid previous stage: %s", previousStage)) } if previousStageOutputID == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } if *previousStageOutputID != "success" { - panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) } if previousStageOutput == nil { - panic(fmt.Errorf("no previous stage output Name")) + panic(fmt.Errorf("no previous stage output ID")) } message := (*previousStageOutput).(map[any]any)["message"].(string) diff --git a/workflow/executor.go b/workflow/executor.go index 02a6462c..f1a4add4 100644 --- a/workflow/executor.go +++ b/workflow/executor.go @@ -58,7 +58,7 @@ type ExecutableWorkflow interface { // Execute runs a workflow until it finishes or until the context expires with the specified input. The input // must only contain primitives (float, int, bool, string, map, slice) and may not contain structs and other - // elements. The output will consist of the output Name, the returned output data corresponding to the output IDs + // elements. The output will consist of the output ID, the returned output data corresponding to the output IDs // schema, or if an error happened, the error. Execute( ctx context.Context, diff --git a/workflow/model.go b/workflow/model.go index 3efe3b22..5ff2b02a 100644 --- a/workflow/model.go +++ b/workflow/model.go @@ -21,7 +21,7 @@ type Workflow struct { // Steps contains the possible steps in this workflow. The data set must contain a valid step structure where the // inputs to stages may consist only of primitive types and expressions. Steps map[string]any `json:"steps"` - // Outputs lets you define one or more outputs. The outputs should be keyed by their output Name (e.g. "success") and + // Outputs lets you define one or more outputs. The outputs should be keyed by their output ID (e.g. "success") and // the value should be the data you wish to output. The data may contain expressions to construct the output. Outputs map[string]any `json:"outputs"` // OutputSchema is an optional override for the automatically inferred output schema from the Outputs data and @@ -182,7 +182,7 @@ type DAGItem struct { StepID string // StageID is the stage of the step provider this item refers to. StageID string - // OutputID is the Name of the output of the step stage. + // OutputID is the ID of the output of the step stage. OutputID string // OutputSchema contains the output-specific schema for this item. OutputSchema schema.StepOutput @@ -212,12 +212,12 @@ func (d DAGItem) String() string { } } -// GetStageNodeID returns the DAG node Name for a stage. +// GetStageNodeID returns the DAG node ID for a stage. func GetStageNodeID(stepID string, stageID string) string { return fmt.Sprintf("steps.%s.%s", stepID, stageID) } -// GetOutputNodeID returns the DAG node Name for a stage output. +// GetOutputNodeID returns the DAG node ID for a stage output. func GetOutputNodeID(stepID string, stageID string, outputID string) string { return fmt.Sprintf("steps.%s.%s.%s", stepID, stageID, outputID) } diff --git a/workflow/workflow.go b/workflow/workflow.go index 4e962576..9ffab54d 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -260,31 +260,31 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo } stageNode, err := l.dag.GetNodeByID(GetStageNodeID(stepID, *previousStage)) if err != nil { - l.logger.Errorf("Failed to get stage node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) - l.recentErrors <- fmt.Errorf("failed to get stage node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.logger.Errorf("Failed to get stage node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.recentErrors <- fmt.Errorf("failed to get stage node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) l.cancel() return } l.logger.Debugf("Removed node '%s' from the DAG", stageNode.ID()) if err := stageNode.Remove(); err != nil { - l.logger.Errorf("Failed to remove stage node Name %s (%w)", stageNode.ID(), err) - l.recentErrors <- fmt.Errorf("failed to remove stage node Name %s (%w)", stageNode.ID(), err) + l.logger.Errorf("Failed to remove stage node ID %s (%w)", stageNode.ID(), err) + l.recentErrors <- fmt.Errorf("failed to remove stage node ID %s (%w)", stageNode.ID(), err) l.cancel() return } if previousStageOutputID != nil { outputNode, err := l.dag.GetNodeByID(GetOutputNodeID(stepID, *previousStage, *previousStageOutputID)) if err != nil { - l.logger.Errorf("Failed to get output node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) - l.recentErrors <- fmt.Errorf("failed to get output node Name %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.logger.Errorf("Failed to get output node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) + l.recentErrors <- fmt.Errorf("failed to get output node ID %s (%w)", GetStageNodeID(stepID, *previousStage), err) l.cancel() return } // Removes the node from the DAG. This results in the nodes not having inbound connections, allowing them to be processed. l.logger.Debugf("Removed node '%s' from the DAG", outputNode.ID()) if err := outputNode.Remove(); err != nil { - l.logger.Errorf("Failed to remove output node Name %s (%w)", outputNode.ID(), err) - l.recentErrors <- fmt.Errorf("failed to remove output node Name %s (%w)", outputNode.ID(), err) + l.logger.Errorf("Failed to remove output node ID %s (%w)", outputNode.ID(), err) + l.recentErrors <- fmt.Errorf("failed to remove output node ID %s (%w)", outputNode.ID(), err) l.cancel() return } @@ -293,7 +293,7 @@ func (l *loopState) onStageComplete(stepID string, previousStage *string, previo if stepLogConfig != nil { l.logger.Writef( stepLogConfig.LogLevel, - "Output Name for step \"%s\" is \"%s\".\nOutput data: \"%s\"", + "Output ID for step \"%s\" is \"%s\".\nOutput data: \"%s\"", stepID, *previousStageOutputID, *previousStageOutput, @@ -362,7 +362,7 @@ func (l *loopState) notifySteps() { //nolint:gocognit // This check is here just to make sure it has the required fields set if node.Item().StepID == "" || node.Item().StageID == "" { // This shouldn't happen - panic("Step or stage Name missing") + panic("Step or stage ID missing") } stageInputData := untypedInputData.(map[any]any) From 30bb6f79c99575e107d9742c5b39d5c3b9ba73cc Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 17:34:25 -0400 Subject: [PATCH 49/54] unfactor more Name to ID --- internal/step/plugin/provider.go | 6 +++--- internal/step/plugin/provider_test.go | 2 +- internal/yaml/parser.go | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index bff40e82..ec7815ba 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -575,10 +575,10 @@ type runningStep struct { runInputAvailable bool logger log.Logger currentStage StageID - runID string // The Name associated with this execution (the workflow step Name) + runID string // The ID associated with this execution (the workflow step ID) deploymentType deployer.DeploymentType source string - pluginStepID string // The Name of the step in the plugin + pluginStepID string // The ID of the step in the plugin state step.RunningStepState useLocalDeployer bool localDeployer deployer.Connector @@ -795,7 +795,7 @@ func (r *runningStep) run() { r.container = container } r.lock.Unlock() - r.logger.Debugf("Successfully deployed container with Name '%s' for step %s/%s", container.ID(), r.runID, r.pluginStepID) + r.logger.Debugf("Successfully deployed container with ID '%s' for step %s/%s", container.ID(), r.runID, r.pluginStepID) if err := r.startStage(container); err != nil { r.startFailed(err) return diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 6d749224..d2ceb17b 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -68,7 +68,7 @@ func (s *startFailStageChangeHandler) OnStepComplete( panic(fmt.Errorf("no previous stage output ID")) } if *previousStageOutputID != "error" { - panic(fmt.Errorf("invalid previous stage output Name: %s", *previousStageOutputID)) + panic(fmt.Errorf("invalid previous stage output ID: %s", *previousStageOutputID)) } if previousStageOutput == nil { panic(fmt.Errorf("no previous stage output ID")) diff --git a/internal/yaml/parser.go b/internal/yaml/parser.go index c602ec1c..9984b525 100644 --- a/internal/yaml/parser.go +++ b/internal/yaml/parser.go @@ -107,7 +107,7 @@ func (n node) Raw() any { } return result default: - panic(fmt.Errorf("bug: unexpected type Name: %s", n.typeID)) + panic(fmt.Errorf("bug: unexpected type ID: %s", n.typeID)) } } From d60986f10f5defefbeb912b5f8160be8369b9954 Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 17:39:32 -0400 Subject: [PATCH 50/54] update to correct kubernetes deployer version --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index c9102a61..032fcef2 100644 --- a/go.mod +++ b/go.mod @@ -10,7 +10,7 @@ require ( go.flow.arcalot.io/deployer v0.4.0 go.flow.arcalot.io/dockerdeployer v0.5.0 go.flow.arcalot.io/expressions v0.2.1 - go.flow.arcalot.io/kubernetesdeployer v0.4.0 + go.flow.arcalot.io/kubernetesdeployer v0.8.0 go.flow.arcalot.io/pluginsdk v0.5.0 go.flow.arcalot.io/podmandeployer v0.6.0 go.flow.arcalot.io/pythondeployer v0.4.0 diff --git a/go.sum b/go.sum index cfcba5ee..1144696a 100644 --- a/go.sum +++ b/go.sum @@ -166,8 +166,8 @@ go.flow.arcalot.io/dockerdeployer v0.5.0 h1:CKsy5K38/0hF6dEry7f0YIfI5uNKGImyCwi8 go.flow.arcalot.io/dockerdeployer v0.5.0/go.mod h1:zE/okmoc47PTdJxiag+Gk9LMzKR1GemeP1Q3woOY3/A= go.flow.arcalot.io/expressions v0.2.1 h1:TAAbDrgJJLpmgA5ASyP/KzrXWtpEaQ8JsCPHgpe5kLw= go.flow.arcalot.io/expressions v0.2.1/go.mod h1:Vw1ScNu4Uyw1/l87LAH8jxe0DyRWwMh+rlfB/BPYDOU= -go.flow.arcalot.io/kubernetesdeployer v0.4.0 h1:y5GaGIsTnnXroqnj8yczM82plQGNttMHVc4N41TCzxQ= -go.flow.arcalot.io/kubernetesdeployer v0.4.0/go.mod h1:XNkG7ARtjdHFigJLLIJ9kcofkq2DzZ8gz2FPCnJON+I= +go.flow.arcalot.io/kubernetesdeployer v0.8.0 h1:UjH/aspPif/k+X65sLWlNDZAW5JlzUfgOnLHOrhxEQk= +go.flow.arcalot.io/kubernetesdeployer v0.8.0/go.mod h1:BhERhKpvQMJkrcW9lbBF4kJEe+OGhz2NpSftZIgtVNQ= go.flow.arcalot.io/pluginsdk v0.5.0 h1:TRS/waCTcdoMZ9neDAcfy3zpzyDnPHRbhV+Y1kpcw3Y= go.flow.arcalot.io/pluginsdk v0.5.0/go.mod h1:2s2f//7uOkBjr1QaiWJD/bqDIeLlINJtD1BhiY4aGPM= go.flow.arcalot.io/podmandeployer v0.6.0 h1:SlcQUU6xt24Oa0OFNnwYE+d+XQbDXErMLvpCi2gMHoA= From 8901b4b1d13e75823d3fb8cd173b7a6ead39c02e Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 18:15:19 -0400 Subject: [PATCH 51/54] fix default value --- internal/step/plugin/provider.go | 2 +- internal/step/plugin/provider_test.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/internal/step/plugin/provider.go b/internal/step/plugin/provider.go index ec7815ba..8912e95b 100644 --- a/internal/step/plugin/provider.go +++ b/internal/step/plugin/provider.go @@ -110,7 +110,7 @@ func (p *pluginProvider) ProviderSchema() map[string]*schema.PropertySchema { nil, nil, nil, - []string{"docker"}, + []string{"image"}, ), }, ), diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index d2ceb17b..7a0724a4 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -404,7 +404,8 @@ func TestProvider_DeployFail(t *testing.T) { stepSchema := map[string]any{ "plugin": map[string]any{ "src": "simulation", - "deployment_type": "builtin"}, + "deployment_type": "builtin", + }, } byteSchema := map[string][]byte{} From e42addfdbdba49b47f4c40a1d35b1ead87e3d734 Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Wed, 1 Nov 2023 18:30:36 -0400 Subject: [PATCH 52/54] Added tests to validate that the workflow fails with invalid deployment configs --- internal/step/plugin/provider_test.go | 36 +++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index 7a0724a4..acaad563 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -164,6 +164,42 @@ func TestProvider_MultipleDeployers(t *testing.T) { assert.Error(t, err) } +func TestProvider_MissingDeployer(t *testing.T) { + logger := log.New( + log.Config{ + Level: log.LevelError, + Destination: log.DestinationStdout, + }, + ) + deployerRegistry := deployer_registry.New() // Empty. So it will error out. + workflowDeployerCfg := map[string]any{ + "builtin": map[string]any{ + "deployer_name": "test-impl", + }, + } + + _, err := plugin.New(logger, deployerRegistry, workflowDeployerCfg) + assert.Error(t, err) +} +func TestProvider_MismatchedDeploymentTypes(t *testing.T) { + logger := log.New( + log.Config{ + Level: log.LevelError, + Destination: log.DestinationStdout, + }, + ) + deployerRegistry := deployer_registry.New(deployer.Any(testdeployer.NewFactory())) + // Mismatched. test-impl is has the deployment type builtin, but we're trying to specify it for the image type. + workflowDeployerCfg := map[string]any{ + "image": map[string]any{ + "deployer_name": "test-impl", + }, + } + + _, err := plugin.New(logger, deployerRegistry, workflowDeployerCfg) + assert.Error(t, err) +} + func TestProvider_Utility(t *testing.T) { workflowDeployerCfg := map[string]any{ "builtin": map[string]any{"deployer_name": "test-impl"}, From ce9c4bbce81bdc5d3d611ffb4e1092e9546ea8bc Mon Sep 17 00:00:00 2001 From: Jared O'Connell Date: Wed, 1 Nov 2023 18:41:23 -0400 Subject: [PATCH 53/54] Bump workflow version, and switch from Python to Docker deployer The docker deployer is already used extensively in the tests. No need to have a second deployer that also needs to be configured --- README.md | 2 +- engine.go | 2 +- engine_test.go | 10 +++++----- internal/step/foreach/provider_example_test.go | 4 ++-- internal/step/plugin/provider_test.go | 8 ++++---- workflow/executor_example_test.go | 2 +- workflow/executor_test.go | 10 +++++----- workflow/workflow_test.go | 16 ++++++++-------- 8 files changed, 27 insertions(+), 27 deletions(-) diff --git a/README.md b/README.md index a5fd2b23..2838b920 100644 --- a/README.md +++ b/README.md @@ -21,7 +21,7 @@ This binary can then be used to run Arcaflow workflows. The simplest workflow is the example plugin workflow using the workflow schema version `v0.1.0`: (save it to workflow.yaml) ```yaml -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: diff --git a/engine.go b/engine.go index 56c484f9..d82687a3 100644 --- a/engine.go +++ b/engine.go @@ -13,7 +13,7 @@ import ( ) var supportedVersions = map[string]struct{}{ - "v0.1.0": {}, + "v0.2.0": {}, } // WorkflowEngine is responsible for executing workflows and returning their result. diff --git a/engine_test.go b/engine_test.go index f697c2e7..be8dd1f0 100644 --- a/engine_test.go +++ b/engine_test.go @@ -14,7 +14,7 @@ import ( ) func TestEngineWorkflow_ParseVersion(t *testing.T) { - _, err := engine.SupportedVersion("v0.1.0") + _, err := engine.SupportedVersion("v0.2.0") assert.NoError(t, err) // test unsupported version @@ -104,7 +104,7 @@ func TestEmptySteps(t *testing.T) { context.Background(), nil, map[string][]byte{ - "workflow.yaml": []byte(`version: v0.1.0 + "workflow.yaml": []byte(`version: v0.2.0 output: [] steps: []`), }, @@ -119,7 +119,7 @@ func TestNoSteps(t *testing.T) { context.Background(), nil, map[string][]byte{ - "workflow.yaml": []byte(`version: v0.1.0 + "workflow.yaml": []byte(`version: v0.2.0 output: []`), }, "", @@ -133,7 +133,7 @@ func TestE2E(t *testing.T) { context.Background(), []byte(`name: Arca Lot`), map[string][]byte{ - "workflow.yaml": []byte(`version: v0.1.0 + "workflow.yaml": []byte(`version: v0.2.0 input: root: RootObject objects: @@ -166,7 +166,7 @@ func TestE2EMultipleOutputs(t *testing.T) { context.Background(), []byte(`name: Arca Lot`), map[string][]byte{ - "workflow.yaml": []byte(`version: v0.1.0 + "workflow.yaml": []byte(`version: v0.2.0 input: root: RootObject objects: diff --git a/internal/step/foreach/provider_example_test.go b/internal/step/foreach/provider_example_test.go index 952cf7ae..6bc248a8 100644 --- a/internal/step/foreach/provider_example_test.go +++ b/internal/step/foreach/provider_example_test.go @@ -16,7 +16,7 @@ import ( // mainWorkflow is the workflow calling the foreach step. var mainWorkflow = ` -version: v0.1.0 +version: v0.2.0 input: root: names objects: @@ -44,7 +44,7 @@ output: ` var subworkflow = ` -version: v0.1.0 +version: v0.2.0 input: root: name objects: diff --git a/internal/step/plugin/provider_test.go b/internal/step/plugin/provider_test.go index acaad563..2f96a2e4 100644 --- a/internal/step/plugin/provider_test.go +++ b/internal/step/plugin/provider_test.go @@ -6,9 +6,9 @@ import ( "go.arcalot.io/log/v2" "go.flow.arcalot.io/deployer" deployer_registry "go.flow.arcalot.io/deployer/registry" + docker "go.flow.arcalot.io/dockerdeployer" "go.flow.arcalot.io/engine/internal/step" "go.flow.arcalot.io/engine/internal/step/plugin" - python "go.flow.arcalot.io/pythondeployer" testdeployer "go.flow.arcalot.io/testdeployer" "sync" "testing" @@ -120,7 +120,7 @@ func TestProvider_MultipleDeployers(t *testing.T) { ) deployerRegistry := deployer_registry.New( deployer.Any(testdeployer.NewFactory()), - deployer.Any(python.NewFactory())) + deployer.Any(docker.NewFactory())) deployTimeMs := 20 workflowDeployerCfg := map[string]any{ "builtin": map[string]any{ @@ -128,8 +128,8 @@ func TestProvider_MultipleDeployers(t *testing.T) { "deploy_time": deployTimeMs, "deploy_succeed": true, }, - "python": map[string]any{ - "deployer_name": "python", + "image": map[string]any{ + "deployer_name": "docker", }, } diff --git a/workflow/executor_example_test.go b/workflow/executor_example_test.go index 25a1011f..b68ab747 100644 --- a/workflow/executor_example_test.go +++ b/workflow/executor_example_test.go @@ -13,7 +13,7 @@ import ( ) var workflowYAML = `--- -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: diff --git a/workflow/executor_test.go b/workflow/executor_test.go index 1b5cd112..daecaf7c 100644 --- a/workflow/executor_test.go +++ b/workflow/executor_test.go @@ -76,7 +76,7 @@ func NewTestImplStepRegistry( } var sharedInputWorkflowYAML = `--- -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -117,7 +117,7 @@ func TestSharedInput(t *testing.T) { } var missingInputWorkflowDefinition1 = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -135,7 +135,7 @@ outputs: ` var missingInputWorkflowDefinition2 = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -161,7 +161,7 @@ func TestMissingInput(t *testing.T) { } var mismatchedStepInputTypesWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -191,7 +191,7 @@ func TestMismatchedStepInputTypes(t *testing.T) { } var mismatchedInputTypesWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: diff --git a/workflow/workflow_test.go b/workflow/workflow_test.go index c94488f2..c2bd6bc4 100644 --- a/workflow/workflow_test.go +++ b/workflow/workflow_test.go @@ -14,7 +14,7 @@ import ( ) var badWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: name objects: @@ -46,7 +46,7 @@ func TestOutputFailed(t *testing.T) { } var stepCancellationWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -94,7 +94,7 @@ func TestStepCancellation(t *testing.T) { } var earlyStepCancellationWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -168,7 +168,7 @@ func TestEarlyStepCancellation(t *testing.T) { } var deploymentStepCancellationWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -236,7 +236,7 @@ func TestDeploymentStepCancellation(t *testing.T) { } var simpleValidLiteralInputWaitWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -267,7 +267,7 @@ func TestSimpleValidWaitWorkflow(t *testing.T) { } var waitForSerialWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -347,7 +347,7 @@ func TestWaitForSerial(t *testing.T) { } var missingInputsFailedDeploymentWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: @@ -390,7 +390,7 @@ func TestMissingInputsFailedDeployment(t *testing.T) { } var missingInputsWrongOutputWorkflowDefinition = ` -version: v0.1.0 +version: v0.2.0 input: root: RootObject objects: From 6cfcfa8ecf3b2fae3ea185be24b943810ed73800 Mon Sep 17 00:00:00 2001 From: Matt Leader Date: Wed, 1 Nov 2023 18:44:16 -0400 Subject: [PATCH 54/54] update readme --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 2838b920..5615a976 100644 --- a/README.md +++ b/README.md @@ -72,7 +72,7 @@ You can load this config by passing the `-config` flag to Arcaflow. ### Supported Workflow Schema Versions -- v0.1.0 +- v0.2.0 ## Deployer options