From 6390a73b948ae231eff4dc766de6857053e749b0 Mon Sep 17 00:00:00 2001 From: William Batista Date: Tue, 20 Aug 2024 19:31:27 -0400 Subject: [PATCH 1/2] move some of plan.go into a proper FSM this doesn't involve too many code changes, more just shifting code around. the reason for doing this is that, if we can move as many partso f deploys as possible into an FSM, it lets us more easily resume deploys that failed, or from midway through a canceled deploy, or anywhere else in a deploy. I want to make more granular FSM transitions for each individual machine too, which I'll either do in a future commit or a future PR. The more individual FSM states we can retry, the more granular 'resuming' we can do. If we can move other parts of deploys into an FSM, it also gives us the ability to resume *those* parts as well. Plus some extra super powers like easily cloning apps, more granular view of deploy failures in a way that we can retry locally, declaratively configuring a fly app, etc. --- go.mod | 1 + go.sum | 2 + .../deploy/machines_deploymachinesapp.go | 18 +- internal/command/deploy/plan.go | 320 ++++++++++-------- internal/command/deploy/plan_test.go | 6 +- 5 files changed, 192 insertions(+), 155 deletions(-) diff --git a/go.mod b/go.mod index 3edf862778..934bff1f68 100644 --- a/go.mod +++ b/go.mod @@ -64,6 +64,7 @@ require ( github.com/prometheus/blackbox_exporter v0.25.0 github.com/prometheus/client_golang v1.19.1 github.com/prometheus/client_model v0.6.1 + github.com/qmuntal/stateless v1.7.0 github.com/r3labs/diff v1.1.0 github.com/samber/lo v1.47.0 github.com/skratchdot/open-golang v0.0.0-20200116055534-eef842397966 diff --git a/go.sum b/go.sum index 0709148cfe..6c16c6035d 100644 --- a/go.sum +++ b/go.sum @@ -548,6 +548,8 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= +github.com/qmuntal/stateless v1.7.0 h1:Gzw/TUfmSQxoof7TSQ4kCa4DYwnDD5szeAI29BAR/jY= +github.com/qmuntal/stateless v1.7.0/go.mod h1:n1HjRBM/cq4uCr3rfUjaMkgeGcd+ykAZwkjLje6jGBM= github.com/r3labs/diff v1.1.0 h1:V53xhrbTHrWFWq3gI4b94AjgEJOerO1+1l0xyHOBi8M= github.com/r3labs/diff v1.1.0/go.mod h1:7WjXasNzi0vJetRcB/RqNl5dlIsmXcTTLmF5IoH6Xig= github.com/rivo/tview v0.0.0-20220307222120-9994674d60a8 h1:xe+mmCnDN82KhC010l3NfYlA8ZbOuzbXAzSYBa6wbMc= diff --git a/internal/command/deploy/machines_deploymachinesapp.go b/internal/command/deploy/machines_deploymachinesapp.go index a698a8e981..e0f4b18cf0 100644 --- a/internal/command/deploy/machines_deploymachinesapp.go +++ b/internal/command/deploy/machines_deploymachinesapp.go @@ -579,7 +579,7 @@ func (md *machineDeployment) updateExistingMachinesWRecovery(ctx context.Context } newAppState := *oldAppState - newAppState.Machines = lo.Map(updateEntries, func(e *machineUpdateEntry, _ int) *fly.Machine { + newAppState.Machines = machineSliceToMap(lo.Map(updateEntries, func(e *machineUpdateEntry, _ int) *fly.Machine { newMach := e.leasableMachine.Machine() if !e.launchInput.SkipLaunch { newMach.State = "started" @@ -590,7 +590,7 @@ func (md *machineDeployment) updateExistingMachinesWRecovery(ctx context.Context } newMach.Config = e.launchInput.Config return newMach - }) + })) switch md.strategy { case "bluegreen": @@ -611,15 +611,19 @@ func (md *machineDeployment) updateExistingMachinesWRecovery(ctx context.Context skipLeaseAcquisition: false, }) case "canary": + var canaryMachID string + + for machID, _ := range oldAppState.Machines { + canaryMachID = machID + break + } + // create a new app state with just a single machine being updated, then the rest of the machines canaryAppState := *oldAppState - canaryAppState.Machines = []*fly.Machine{oldAppState.Machines[0]} + canaryAppState.Machines = machineSliceToMap([]*fly.Machine{oldAppState.Machines[canaryMachID]}) newCanaryAppState := newAppState - canaryMach, _ := lo.Find(newAppState.Machines, func(m *fly.Machine) bool { - return m.ID == oldAppState.Machines[0].ID - }) - newCanaryAppState.Machines = []*fly.Machine{canaryMach} + newCanaryAppState.Machines = machineSliceToMap([]*fly.Machine{newAppState.Machines[canaryMachID]}) if err := md.updateMachinesWRecovery(ctx, &canaryAppState, &newCanaryAppState, nil, updateMachineSettings{ pushForward: true, diff --git a/internal/command/deploy/plan.go b/internal/command/deploy/plan.go index a66e45b0b2..919c89555b 100644 --- a/internal/command/deploy/plan.go +++ b/internal/command/deploy/plan.go @@ -4,11 +4,13 @@ import ( "context" "errors" "fmt" + "reflect" "strings" "sync" "time" "github.com/google/go-cmp/cmp" + "github.com/qmuntal/stateless" "github.com/samber/lo" fly "github.com/superfly/fly-go" "github.com/superfly/flyctl/internal/ctrlc" @@ -51,8 +53,30 @@ func (m *MachineLogger) getLoggerFromID(id string) statuslogger.StatusLine { return m.store[id] } +type FSMState string + +const ( + updateNotStarted FSMState = "updateNotStarted" + + updatingMachines FSMState = "updatingMachines" + failedUpdatingMachines FSMState = "failedUpdatingMachines" + updatedMachines FSMState = "updatedMachines" + + updateComplete FSMState = "updateComplete" + updateFailed FSMState = "updateFailed" +) + +type fsmTrigger string + +const ( + triggerUpdateMachines fsmTrigger = "triggerUpdateMachines" + triggerUpdateMachinesFailed fsmTrigger = "triggerUpdateMachinesFailed" + triggerUpdateFailed fsmTrigger = "triggerUpdateFailed" + triggerUpdateComplete fsmTrigger = "triggerUpdateComplete" +) + type AppState struct { - Machines []*fly.Machine + Machines map[string]*fly.Machine } type machinePairing struct { @@ -70,11 +94,10 @@ func (md *machineDeployment) appState(ctx context.Context, existingAppState *App return nil, err } + // we need to copy any leases we might have acquired if existingAppState != nil { for _, machine := range machines { - if existingMachine, ok := lo.Find(existingAppState.Machines, func(m *fly.Machine) bool { - return m.ID == machine.ID - }); ok { + if existingMachine, ok := existingAppState.Machines[machine.ID]; ok { machine.LeaseNonce = existingMachine.LeaseNonce } } @@ -82,12 +105,18 @@ func (md *machineDeployment) appState(ctx context.Context, existingAppState *App // TODO: could this be a list of machine id -> config? appState := &AppState{ - Machines: machines, + Machines: machineSliceToMap(machines), } return appState, nil } +func machineSliceToMap(machs []*fly.Machine) map[string]*fly.Machine { + return lo.KeyBy(machs, func(mach *fly.Machine) string { + return mach.ID + }) +} + type healthcheckResult struct { regularChecksPassed bool smokeChecksPassed bool @@ -164,166 +193,155 @@ func (md *machineDeployment) updateMachinesWRecovery(ctx context.Context, oldApp sl, ) - machineLogger.initFromMachinePairs(machineTuples) + poolSize := md.getPoolSize(len(machineTuples)) - machPairByProcessGroup := lo.GroupBy(machineTuples, func(machPair machinePairing) string { - if machPair.oldMachine != nil { - return machPair.oldMachine.ProcessGroup() - } else if machPair.newMachine != nil { - return machPair.newMachine.ProcessGroup() - } else { - return "" - } - }) + machineLogger.initFromMachinePairs(machineTuples) - poolSize := md.getPoolSize(len(machineTuples)) + appStateFSM := stateless.NewStateMachine(updateNotStarted) + appStateFSM.SetTriggerParameters(triggerUpdateMachines, reflect.TypeOf(machineTuples), reflect.TypeOf(poolSize)) - if !settings.skipLeaseAcquisition { - attempts := 0 + appStateFSM.Configure(updateNotStarted).Permit(triggerUpdateMachines, updatedMachines) + appStateFSM.Configure(updatingMachines).OnEntry(func(ctx context.Context, args ...any) error { + machPairs := args[0].([]machinePairing) + poolSize := args[1].(int) - defer func() { - span.SetAttributes(attribute.Int("lease_attempts", attempts)) - }() + return md.acquireLeases(ctx, machPairs, poolSize, machineLogger) + }).OnExit(func(ctx context.Context, args ...any) error { + machPairs := args[0].([]machinePairing) + return md.releaseLeases(ctx, machPairs, machineLogger) + }) - for { - err := md.acquireLeases(ctx, machineTuples, poolSize, machineLogger) - if err == nil { - break - } - attempts += 1 + attempts := 0 + updateErrors := []error{} + appStateFSM.Configure(failedUpdatingMachines).SubstateOf(updatingMachines).Permit(triggerUpdateFailed, updateFailed).Permit(triggerUpdateMachines, updatedMachines).OnEntry(func(_ context.Context, args ...any) error { + err := args[0].(error) + updateErrors = append(updateErrors, err) - var unrecoverableErr *unrecoverableError - if attempts > md.deployRetries || errors.As(err, &unrecoverableErr) || errors.Is(err, context.Canceled) { - span.RecordError(err) - return fmt.Errorf("failed to acquire leases: %w", err) - } + switch { + case isUnrecoverableErr(err): + return err + case attempts > md.deployRetries: + return err + default: + fmt.Fprintln(md.io.ErrOut, "Failed to update machines:", err, "Retrying...") + time.Sleep(1 * time.Second) + return nil } - - defer func() { - err := md.releaseLeases(ctx, machineTuples, machineLogger) - if err != nil { - fmt.Fprintln(md.io.ErrOut, "Failed to release leases:", err) - span.RecordError(err) + }).OnExit(func(ctx context.Context, args ...any) (err error) { + attempts += 1 + oldAppState, err = md.appState(ctx, oldAppState) + return err + }) + appStateFSM.Configure(updatedMachines).SubstateOf(updatingMachines).Permit(triggerUpdateComplete, updateComplete).Permit(triggerUpdateMachinesFailed, failedUpdatingMachines).Permit(triggerUpdateFailed, updateFailed).OnEntryFrom(triggerUpdateMachines, func(_ context.Context, args ...any) error { + machinePairs := args[0].([]machinePairing) + machPairByProcessGroup := lo.GroupBy(machinePairs, func(machPair machinePairing) string { + if machPair.newMachine != nil { + return machPair.newMachine.ProcessGroup() + } else if machPair.oldMachine != nil { + return machPair.oldMachine.ProcessGroup() + } else { + return "" } - }() - } - - pgroup := errgroup.Group{} - pgroup.SetLimit(rollingStrategyMaxConcurrentGroups) - - // We want to update by process group - for _, machineTuples := range machPairByProcessGroup { - machineTuples := machineTuples - pgroup.Go(func() error { - eg, ctx := errgroup.WithContext(ctx) - - warmMachines := lo.Filter(machineTuples, func(e machinePairing, i int) bool { - if e.oldMachine != nil && e.oldMachine.State == "started" { - return true - } - if e.newMachine != nil && e.newMachine.State == "started" { - return true - } - return false - }) - - coldMachines := lo.Filter(machineTuples, func(e machinePairing, i int) bool { - if e.oldMachine != nil && e.oldMachine.State != "started" { - return true - } - if e.newMachine != nil && e.newMachine.State != "started" { - return true - } - return false - }) - - eg.Go(func() (err error) { - poolSize := len(coldMachines) - if poolSize >= STOPPED_MACHINES_POOL_SIZE { - poolSize = STOPPED_MACHINES_POOL_SIZE - } + }) - if len(coldMachines) > 0 { - // for cold machines, we can update all of them at once. - // there's no need for protection against downtime since the machines are already stopped - return md.updateProcessGroup(ctx, coldMachines, machineLogger, poolSize) + pgroup := errgroup.Group{} + pgroup.SetLimit(rollingStrategyMaxConcurrentGroups) + + // We want to update by process group + for _, machineTuples := range machPairByProcessGroup { + machineTuples := machineTuples + pgroup.Go(func() error { + eg, ctx := errgroup.WithContext(ctx) + + warmMachines := lo.Filter(machineTuples, func(e machinePairing, i int) bool { + if e.oldMachine != nil && e.oldMachine.State == "started" { + return true + } + if e.newMachine != nil && e.newMachine.State == "started" { + return true + } + return false + }) + + coldMachines := lo.Filter(machineTuples, func(e machinePairing, i int) bool { + if e.oldMachine != nil && e.oldMachine.State != "started" { + return true + } + if e.newMachine != nil && e.newMachine.State != "started" { + return true + } + return false + }) + + eg.Go(func() (err error) { + poolSize := len(coldMachines) + if poolSize >= STOPPED_MACHINES_POOL_SIZE { + poolSize = STOPPED_MACHINES_POOL_SIZE + } + + if len(coldMachines) > 0 { + // for cold machines, we can update all of them at once. + // there's no need for protection against downtime since the machines are already stopped + return md.updateProcessGroup(ctx, coldMachines, machineLogger, poolSize) + } + + return nil + }) + + eg.Go(func() (err error) { + // for warm machines, we update them in chunks of size, md.maxUnavailable. + // this is to prevent downtime/low-latency during deployments + poolSize := md.getPoolSize(len(warmMachines)) + if len(warmMachines) > 0 { + return md.updateProcessGroup(ctx, warmMachines, machineLogger, poolSize) + } + return nil + }) + + err := eg.Wait() + if err != nil { + span.RecordError(err) + if strings.Contains(err.Error(), "lease currently held by") { + err = &unrecoverableError{err: err} + } + return err } return nil }) - - eg.Go(func() (err error) { - // for warm machines, we update them in chunks of size, md.maxUnavailable. - // this is to prevent downtime/low-latency during deployments - poolSize := md.getPoolSize(len(warmMachines)) - if len(warmMachines) > 0 { - return md.updateProcessGroup(ctx, warmMachines, machineLogger, poolSize) - } - return nil - }) - - err := eg.Wait() - if err != nil { - span.RecordError(err) - if strings.Contains(err.Error(), "lease currently held by") { - err = &unrecoverableError{err: err} - } - return err - } - - return nil - }) - } - - attempts := 0 - if updateErr := pgroup.Wait(); updateErr != nil { - var unrecoverableErr *unrecoverableError - if !settings.pushForward || errors.As(updateErr, &unrecoverableErr) || errors.Is(updateErr, context.Canceled) { - span.RecordError(updateErr) - return updateErr } - // if we fail to update the machines, we should push the state forward if possible - for { - defer func() { - span.SetAttributes(attribute.Int("update_attempts", attempts)) - }() + return pgroup.Wait() + }) - if attempts > md.deployRetries { - fmt.Fprintln(md.io.ErrOut, "Failed to update machines:", updateErr) - span.RecordError(updateErr) - return updateErr - } + appStateFSM.Configure(updateFailed).OnEntry(func(ctx context.Context, args ...any) error { + err := args[0].(error) + span.SetAttributes(attribute.Int("update_attempts", attempts)) + span.SetAttributes(attribute.StringSlice("update_errors", lo.Map(updateErrors, func(err error, _ int) string { + return err.Error() + }))) - currentState, err := md.appState(ctx, oldAppState) - if err != nil { - span.RecordError(updateErr) - return fmt.Errorf("failed to get current app state: %w", err) - } - err = md.updateMachinesWRecovery(ctx, currentState, newAppState, sl, updateMachineSettings{ - pushForward: false, - skipHealthChecks: settings.skipHealthChecks, - skipSmokeChecks: settings.skipSmokeChecks, - skipLeaseAcquisition: true, - }) - if err == nil { - break - } else if errors.Is(err, context.Canceled) { - span.RecordError(updateErr) - return err - } else { - if errors.As(err, &unrecoverableErr) { - span.RecordError(updateErr) - return err - } - fmt.Fprintln(md.io.ErrOut, "Failed to update machines:", err, "Retrying...") + span.RecordError(err) + return err + }) + appStateFSM.Configure(updateComplete).OnEntry(func(ctx context.Context, args ...any) error { + span.SetAttributes(attribute.Int("update_attempts", attempts)) + span.SetAttributes(attribute.StringSlice("update_errors", lo.Map(updateErrors, func(err error, _ int) string { + return err.Error() + }))) + return nil + }) + + for { + if updateErr := appStateFSM.FireCtx(ctx, triggerUpdateMachines, machineTuples, poolSize); updateErr != nil { + // if we return an error when triggering machines failed, it means we're done + if err := appStateFSM.FireCtx(ctx, triggerUpdateMachinesFailed, updateErr); err != nil { + return appStateFSM.FireCtx(ctx, triggerUpdateFailed) } - attempts += 1 - time.Sleep(1 * time.Second) + } else { + return appStateFSM.FireCtx(ctx, triggerUpdateComplete, machineTuples) } } - - return nil } func (md *machineDeployment) updateProcessGroup(ctx context.Context, machineTuples []machinePairing, machineLogger *MachineLogger, poolSize int) error { @@ -770,3 +788,15 @@ func (md *machineDeployment) createMachine(ctx context.Context, machConfig *fly. return machine, nil } + +func isUnrecoverableErr(err error) bool { + var unrecoverableErr *unrecoverableError + switch { + case errors.As(err, &unrecoverableErr): + return true + case errors.Is(err, context.Canceled): + return true + default: + return false + } +} diff --git a/internal/command/deploy/plan_test.go b/internal/command/deploy/plan_test.go index 10cb12edda..0b5fa79338 100644 --- a/internal/command/deploy/plan_test.go +++ b/internal/command/deploy/plan_test.go @@ -73,7 +73,7 @@ func TestAppState(t *testing.T) { appState, error := md.appState(ctx, nil) assert.NoError(t, error) - assert.Equal(t, appState.Machines, machines) + assert.Equal(t, appState.Machines, machineSliceToMap(machines)) } @@ -293,10 +293,10 @@ func TestUpdateMachines(t *testing.T) { } oldAppState := &AppState{ - Machines: oldMachines, + Machines: machineSliceToMap(oldMachines), } newAppState := &AppState{ - Machines: newMachines, + Machines: machineSliceToMap(newMachines), } settings := updateMachineSettings{ pushForward: true, From 60a71983980e3addb4f34586eb9f0b5728ba4142 Mon Sep 17 00:00:00 2001 From: William Batista Date: Thu, 22 Aug 2024 10:04:53 -0400 Subject: [PATCH 2/2] amend this commit plz --- internal/build/imgsrc/depot.go | 1 + internal/command/deploy/plan.go | 56 +++++++++++++++++++++++---------- 2 files changed, 41 insertions(+), 16 deletions(-) diff --git a/internal/build/imgsrc/depot.go b/internal/build/imgsrc/depot.go index 0299c843ee..8776eb2070 100644 --- a/internal/build/imgsrc/depot.go +++ b/internal/build/imgsrc/depot.go @@ -116,6 +116,7 @@ func depotBuild(ctx context.Context, streams *iostreams.IOStreams, opts ImageOpt streams.StopProgressIndicator() return nil, buildErr } + buildState.BuilderMeta.DepotBuildId = build.ID defer func() { buildkit.Release() build.Finish(buildErr) diff --git a/internal/command/deploy/plan.go b/internal/command/deploy/plan.go index 919c89555b..23403db6d9 100644 --- a/internal/command/deploy/plan.go +++ b/internal/command/deploy/plan.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "reflect" "strings" "sync" "time" @@ -103,7 +102,6 @@ func (md *machineDeployment) appState(ctx context.Context, existingAppState *App } } - // TODO: could this be a list of machine id -> config? appState := &AppState{ Machines: machineSliceToMap(machines), } @@ -198,17 +196,12 @@ func (md *machineDeployment) updateMachinesWRecovery(ctx context.Context, oldApp machineLogger.initFromMachinePairs(machineTuples) appStateFSM := stateless.NewStateMachine(updateNotStarted) - appStateFSM.SetTriggerParameters(triggerUpdateMachines, reflect.TypeOf(machineTuples), reflect.TypeOf(poolSize)) appStateFSM.Configure(updateNotStarted).Permit(triggerUpdateMachines, updatedMachines) appStateFSM.Configure(updatingMachines).OnEntry(func(ctx context.Context, args ...any) error { - machPairs := args[0].([]machinePairing) - poolSize := args[1].(int) - - return md.acquireLeases(ctx, machPairs, poolSize, machineLogger) + return md.acquireLeases(ctx, machineTuples, poolSize, machineLogger) }).OnExit(func(ctx context.Context, args ...any) error { - machPairs := args[0].([]machinePairing) - return md.releaseLeases(ctx, machPairs, machineLogger) + return md.releaseLeases(ctx, machineTuples, machineLogger) }) attempts := 0 @@ -333,7 +326,7 @@ func (md *machineDeployment) updateMachinesWRecovery(ctx context.Context, oldApp }) for { - if updateErr := appStateFSM.FireCtx(ctx, triggerUpdateMachines, machineTuples, poolSize); updateErr != nil { + if updateErr := appStateFSM.FireCtx(ctx, triggerUpdateMachines); updateErr != nil { // if we return an error when triggering machines failed, it means we're done if err := appStateFSM.FireCtx(ctx, triggerUpdateMachinesFailed, updateErr); err != nil { return appStateFSM.FireCtx(ctx, triggerUpdateFailed) @@ -518,14 +511,45 @@ func compareConfigs(ctx context.Context, oldConfig, newConfig *fly.MachineConfig return isEqual } -func (md *machineDeployment) updateMachineWChecks(ctx context.Context, oldMachine, newMachine *fly.Machine, sl statuslogger.StatusLine, io *iostreams.IOStreams, healthcheckResult *healthcheckResult) error { - ctx, span := tracing.GetTracer().Start(ctx, "update_machine_w_checks", trace.WithAttributes( - attribute.Bool("smoke_checks", healthcheckResult.smokeChecksPassed), - attribute.Bool("machine_checks", healthcheckResult.machineChecksPassed), - attribute.Bool("regular_checks", healthcheckResult.regularChecksPassed), - )) +type machineFsmState string + +const ( + machineUpdateNotStarted machineFsmState = "machineUpdateNotStarted" + machineCreated machineFsmState = "machineCreated" + machineReplaced machineFsmState = "machineReplaced" + machineDestroyed machineFsmState = "machineDestroyed" + machineConfigUpdateFailedBadFlaps machineFsmState = "machineConfigUpdateFailedBadFlaps " + machineConfigUpdated machineFsmState = "machineUpdated" + machineGoodStatus machineFsmState = "machineGoodStatus" + machineGoodSmokeChecks machineFsmState = "machineGoodSmokeChecks" + machineGoodMachineChecks machineFsmState = "machineGoodSmokeChecks" + machineGoodHealthChecks machineFsmState = "machineGoodHealthChecks" + machineUpdateComplete machineFsmState = "machineUpdateComplete" +) + +type machineFsmTrigger string + +const ( + triggerMachineEnsureCreated machineFsmState = "triggerMachineCreated" + triggerMachineConfigUpdated machineFsmState = "triggerMachineConfigUpdated" + triggerMachineGoodStatus machineFsmState = "triggerMachineGoodStatus" + triggerMachineGoodSmokeChecks machineFsmState = "triggerMachineGoodSmokeChecks" + triggerMachineGoodMachineChecks machineFsmState = "triggerMachineGoodMachineChecks" + triggerMachineGoodHealthChecks machineFsmState = "triggerMachineGoodHealthChecks" + triggerMachineUpdateComplete machineFsmState = "triggerMachineUpdateComplete" +) + +func (md *machineDeployment) updateMachineWChecks(ctx context.Context, oldMachine, newMachine *fly.Machine, sl statuslogger.StatusLine, io *iostreams.IOStreams) error { + ctx, span := tracing.GetTracer().Start(ctx, "update_machine_w_checks") defer span.End() + machineStateFSM := stateless.NewStateMachine(machineUpdateNotStarted) + machineStateFSM.Configure(machineUpdateNotStarted).Permit(triggerMachineEnsureCreated, machineCreated) + machineStateFSM.Configure(machineCreated).OnEntry(func(ctx context.Context, args ...any) error { + + return nil + }) + var machine *fly.Machine var lease *fly.MachineLease