From 817ae4b6f6a9c49c5b76b81a161cb9ce8a22fcb9 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 22 Sep 2022 20:14:03 -0500 Subject: [PATCH 01/40] fast cache working-ish Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 66 +++++++++++ pkg/controller/nodes/dynamic/handler.go | 2 +- pkg/controller/nodes/executor.go | 95 +++++++++++++++- pkg/controller/nodes/handler/iface.go | 10 ++ pkg/controller/nodes/handler_factory.go | 2 +- pkg/controller/nodes/task/handler.go | 30 ++--- .../nodes/task/pre_post_execution.go | 106 ++++++++++++++++-- 7 files changed, 282 insertions(+), 29 deletions(-) create mode 100644 pkg/controller/nodes/cache.go diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go new file mode 100644 index 000000000..f63dcc612 --- /dev/null +++ b/pkg/controller/nodes/cache.go @@ -0,0 +1,66 @@ +package nodes + +import ( + "context" + + "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + + "github.com/flyteorg/flytestdlib/logger" + + "github.com/pkg/errors" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + /*"github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" + + "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + //pluginCore "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/core" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" + "github.com/flyteorg/flytestdlib/logger" + "github.com/pkg/errors" + //"google.golang.org/grpc/codes" + //"google.golang.org/grpc/status" + + "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" + errors2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors"*/ +) + +func (n *nodeExecutor) CheckCacheCatalog(ctx context.Context, key catalog.Key) (catalog.Entry, error) { + //logger.Infof(ctx, "Catalog CacheEnabled: Looking up catalog Cache.") + resp, err := n.catalog.Get(ctx, key) + if err != nil { + causeErr := errors.Cause(err) + if taskStatus, ok := status.FromError(causeErr); ok && taskStatus.Code() == codes.NotFound { + //t.metrics.catalogMissCount.Inc(ctx) + logger.Infof(ctx, "Catalog CacheMiss: Artifact not found in Catalog. Executing Task.") + return catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_MISS, nil)), nil + } + + //t.metrics.catalogGetFailureCount.Inc(ctx) + logger.Errorf(ctx, "Catalog Failure: memoization check failed. err: %v", err.Error()) + return catalog.Entry{}, errors.Wrapf(err, "Failed to check Catalog for previous results") + } + + if resp.GetStatus().GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT { + logger.Errorf(ctx, "No CacheHIT and no Error received. Illegal state, Cache State: %s", resp.GetStatus().GetCacheStatus().String()) + // TODO should this be an error? + return resp, nil + } + + // TODO @hamersaw - do we need this?!?! + //logger.Infof(ctx, "Catalog CacheHit: for task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) + //t.metrics.catalogHitCount.Inc(ctx) + /*if iface := tk.Interface; iface != nil && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 { + if err := outputWriter.Put(ctx, resp.GetOutputs()); err != nil { + logger.Errorf(ctx, "failed to write data to Storage, err: %v", err.Error()) + return catalog.Entry{}, errors.Wrapf(err, "failed to copy cached results for task.") + } + }*/ + // SetCached. + return resp, nil +} diff --git a/pkg/controller/nodes/dynamic/handler.go b/pkg/controller/nodes/dynamic/handler.go index fc2a4cb57..d1fd89ba2 100644 --- a/pkg/controller/nodes/dynamic/handler.go +++ b/pkg/controller/nodes/dynamic/handler.go @@ -33,7 +33,7 @@ import ( const dynamicNodeID = "dynamic-node" type TaskNodeHandler interface { - handler.Node + handler.CacheableNode ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader, m catalog.Metadata) (catalog.Status, *io.ExecutionError, error) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index f26b7d2cf..29869808a 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -102,6 +102,8 @@ type nodeExecutor struct { recoveryClient recovery.Client eventConfig *config.EventConfig clusterID string + catalog catalog.Client + asyncCatalog catalog.AsyncClient } func (c *nodeExecutor) RecordTransitionLatency(ctx context.Context, dag executors.DAGStructure, nl executors.NodeLookup, node v1alpha1.ExecutableNode, nodeStatus v1alpha1.ExecutableNodeStatus) { @@ -453,7 +455,9 @@ func (c *nodeExecutor) finalize(ctx context.Context, h handler.Node, nCtx handle return h.Finalize(ctx, nCtx) } -func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executors.DAGStructure, nCtx *nodeExecContext, _ handler.Node) (executors.NodeStatus, error) { +func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executors.DAGStructure, nCtx *nodeExecContext, h handler.Node) (executors.NodeStatus, error) { + nodeStatus := nCtx.NodeStatus() + logger.Debugf(ctx, "Node not yet started, running pre-execute") defer logger.Debugf(ctx, "Node pre-execute completed") p, err := c.preExecute(ctx, dag, nCtx) @@ -470,12 +474,88 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusPending, nil } + // TODO @hamersaw + if cacheHandler, ok := h.(handler.CacheableNode); ok { + cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return executors.NodeStatusUndefined, err + } + + if cacheable { + cacheCatalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) + if err != nil { + // TODO @hamersaw fail + return executors.NodeStatusUndefined, err + } + + entry, err := c.CheckCacheCatalog(ctx, cacheCatalogKey) + if err != nil { + // TODO @hamersaw fail + return executors.NodeStatusUndefined, err + } + + // TODO @hamersaw - figure out + if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + // copy cached outputs to node outputs + o, ee, err := entry.GetOutputs().Read(ctx) + if err != nil { + logger.Errorf(ctx, "failed to read from catalog, err: %s", err.Error()) + return executors.NodeStatusUndefined, err + } + + if ee != nil { + logger.Errorf(ctx, "got execution error from catalog output reader? This should not happen, err: %s", ee.String()) + return executors.NodeStatusUndefined, errors.Errorf(errors.IllegalStateError, nCtx.NodeID(), "execution error from a cache output, bad state: %s", ee.String()) + } + + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + if err := nCtx.DataStore().WriteProtobuf(ctx, outputFile, storage.Options{}, o); err != nil { + logger.Errorf(ctx, "failed to write cached value to datastore, err: %s", err.Error()) + return executors.NodeStatusUndefined, err + } + + // update NodeStatus to Success + nodeStatus.ClearSubNodeStatus() + nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) + + // TODO send events? (with cache hit) - for some reason this is not showing up as CACHE_HIT in UI + phaseInfo := handler.PhaseInfoSuccess(&handler.ExecutionInfo{ + OutputInfo: &handler.OutputInfo { + OutputURI: outputFile, + }, + TaskNodeInfo: &handler.TaskNodeInfo { + TaskNodeMetadata: &event.TaskNodeMetadata{ + CacheStatus: entry.GetStatus().GetCacheStatus(), + CatalogKey: entry.GetStatus().GetMetadata(), + }, + }, + }) + + nev, err := ToNodeExecutionEvent(nCtx.NodeExecutionMetadata().GetNodeExecutionID(), + phaseInfo, nCtx.InputReader().GetInputPath().String(), nodeStatus, nCtx.ExecutionContext().GetEventVersion(), + nCtx.ExecutionContext().GetParentInfo(), nCtx.node, c.clusterID, nCtx.NodeStateReader().GetDynamicNodeState().Phase) + if err != nil { + return executors.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "could not convert phase info to event") + } + err = c.IdempotentRecordEvent(ctx, nev) + if err != nil { + logger.Warningf(ctx, "Failed to record nodeEvent, error [%s]", err.Error()) + return executors.NodeStatusUndefined, errors.Wrapf(errors.EventRecordingFailed, nCtx.NodeID(), err, "failed to record node event") + } + + // TODO return handleDownstream() + return executors.NodeStatusSuccess, nil + //return c.handleDownstream(ctx, execContext, dag, nl, currentNode) + } + } + } + np, err := ToNodePhase(p.GetPhase()) if err != nil { return executors.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "failed to move from queued") } - nodeStatus := nCtx.NodeStatus() if np != nodeStatus.GetPhase() { // assert np == Queued! logger.Infof(ctx, "Change in node state detected from [%s] -> [%s]", nodeStatus.GetPhase().String(), np.String()) @@ -1141,6 +1221,15 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora return nil, err } + async, err := catalog.NewAsyncClient(catalogClient, *catalog.GetConfig(), scope.NewSubScope("async_catalog")) + if err != nil { + return nil, err + } + + if err = async.Start(ctx); err != nil { + return nil, err + } + nodeScope := scope.NewSubScope("node") exec := &nodeExecutor{ store: store, @@ -1180,6 +1269,8 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora recoveryClient: recoveryClient, eventConfig: eventConfig, clusterID: clusterID, + catalog: catalogClient, + asyncCatalog: async, } nodeHandlerFactory, err := NewHandlerFactory(ctx, exec, workflowLauncher, launchPlanReader, kubeClient, catalogClient, recoveryClient, eventConfig, clusterID, nodeScope) exec.nodeHandlerFactory = nodeHandlerFactory diff --git a/pkg/controller/nodes/handler/iface.go b/pkg/controller/nodes/handler/iface.go index d0b359171..8dd752b0e 100644 --- a/pkg/controller/nodes/handler/iface.go +++ b/pkg/controller/nodes/handler/iface.go @@ -2,6 +2,8 @@ package handler import ( "context" + + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" ) //go:generate mockery -all -case=underscore @@ -24,3 +26,11 @@ type Node interface { // It is guaranteed that Handle -> (happens before) -> Finalize. Abort -> finalize may be repeated multiple times Finalize(ctx context.Context, executionContext NodeExecutionContext) error } + +// TODO @hamersaw document +type CacheableNode interface { + Node + GetCatalogKey(ctx context.Context, executionContext NodeExecutionContext) (catalog.Key, error) + IsCacheable(ctx context.Context, executionContext NodeExecutionContext) (bool, error) + IsCacheSerializable(ctx context.Context, executionContext NodeExecutionContext) (bool, error) +} diff --git a/pkg/controller/nodes/handler_factory.go b/pkg/controller/nodes/handler_factory.go index 72e361070..237e2459c 100644 --- a/pkg/controller/nodes/handler_factory.go +++ b/pkg/controller/nodes/handler_factory.go @@ -58,7 +58,7 @@ func NewHandlerFactory(ctx context.Context, executor executors.Node, workflowLau launchPlanReader launchplan.Reader, kubeClient executors.Client, client catalog.Client, recoveryClient recovery.Client, eventConfig *config.EventConfig, clusterID string, scope promutils.Scope) (HandlerFactory, error) { - t, err := task.New(ctx, kubeClient, client, eventConfig, clusterID, scope) + t, err := task.New(ctx, kubeClient, client ,eventConfig, clusterID, scope) if err != nil { return nil, err } diff --git a/pkg/controller/nodes/task/handler.go b/pkg/controller/nodes/task/handler.go index ec58c28f7..25888cd3a 100644 --- a/pkg/controller/nodes/task/handler.go +++ b/pkg/controller/nodes/task/handler.go @@ -46,17 +46,17 @@ const pluginContextKey = contextutils.Key("plugin") type metrics struct { pluginPanics labeled.Counter unsupportedTaskType labeled.Counter - catalogPutFailureCount labeled.Counter + /*catalogPutFailureCount labeled.Counter catalogGetFailureCount labeled.Counter catalogPutSuccessCount labeled.Counter catalogMissCount labeled.Counter - catalogHitCount labeled.Counter + catalogHitCount labeled.Counter*/ pluginExecutionLatency labeled.StopWatch pluginQueueLatency labeled.StopWatch - reservationGetSuccessCount labeled.Counter + /*reservationGetSuccessCount labeled.Counter reservationGetFailureCount labeled.Counter reservationReleaseSuccessCount labeled.Counter - reservationReleaseFailureCount labeled.Counter + reservationReleaseFailureCount labeled.Counter*/ // TODO We should have a metric to capture custom state size scope promutils.Scope @@ -513,10 +513,11 @@ func (t Handler) Handle(ctx context.Context, nCtx handler.NodeExecutionContext) return handler.UnknownTransition, errors.Wrapf(errors.UnsupportedTaskTypeError, nCtx.NodeID(), err, "unable to resolve plugin") } - checkCatalog := !p.GetProperties().DisableNodeLevelCaching + // TODO @hamersaw remove + /*checkCatalog := !p.GetProperties().DisableNodeLevelCaching if !checkCatalog { logger.Infof(ctx, "Node level caching is disabled. Skipping catalog read.") - } + }*/ tCtx, err := t.newTaskExecutionContext(ctx, nCtx, p) if err != nil { @@ -533,7 +534,8 @@ func (t Handler) Handle(ctx context.Context, nCtx handler.NodeExecutionContext) // So now we will derive this from the plugin phase // TODO @kumare re-evaluate this decision - // STEP 1: Check Cache + // TODO @hamersaw remove + /*// STEP 1: Check Cache if (ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache) && checkCatalog { // This is assumed to be first time. we will check catalog and call handle entry, err := t.CheckCatalogCache(ctx, tCtx.tr, nCtx.InputReader(), tCtx.ow) @@ -602,7 +604,7 @@ func (t Handler) Handle(ctx context.Context, nCtx handler.NodeExecutionContext) return pluginTrns.FinalTransition(ctx) } } - } + }*/ barrierTick := uint32(0) // STEP 2: If no cache-hit and not transitioning to PhaseWaitingForCache, then lets invoke the plugin and wait for a transition out of undefined @@ -818,11 +820,11 @@ func (t Handler) Finalize(ctx context.Context, nCtx handler.NodeExecutionContext }() // release catalog reservation (if exists) - ownerID := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetGeneratedName() + /*ownerID := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetGeneratedName() _, err = t.ReleaseCatalogReservation(ctx, ownerID, tCtx.tr, tCtx.InputReader()) if err != nil { return errors.Wrapf(errors.CatalogCallFailed, nCtx.NodeID(), err, "failed to release reservation") - } + }*/ childCtx := context.WithValue(ctx, pluginContextKey, p.GetID()) err = p.Finalize(childCtx, tCtx) @@ -850,17 +852,17 @@ func New(ctx context.Context, kubeClient executors.Client, client catalog.Client metrics: &metrics{ pluginPanics: labeled.NewCounter("plugin_panic", "Task plugin paniced when trying to execute a Handler.", scope), unsupportedTaskType: labeled.NewCounter("unsupported_tasktype", "No Handler plugin configured for Handler type", scope), - catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", scope), + /*catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", scope), catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", scope), catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", scope), catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", scope), - catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", scope), + catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", scope),*/ pluginExecutionLatency: labeled.NewStopWatch("plugin_exec_latency", "Time taken to invoke plugin for one round", time.Microsecond, scope), pluginQueueLatency: labeled.NewStopWatch("plugin_queue_latency", "Time spent by plugin in queued phase", time.Microsecond, scope), - reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", scope), + /*reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", scope), reservationGetSuccessCount: labeled.NewCounter("reservation_get_success_count", "Reservation GetOrExtend success count", scope), reservationReleaseFailureCount: labeled.NewCounter("reservation_release_failure_count", "Reservation Release failure count", scope), - reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", scope), + reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", scope),*/ scope: scope, }, pluginScope: scope.NewSubScope("plugin"), diff --git a/pkg/controller/nodes/task/pre_post_execution.go b/pkg/controller/nodes/task/pre_post_execution.go index 1a6903b76..f3b51ff04 100644 --- a/pkg/controller/nodes/task/pre_post_execution.go +++ b/pkg/controller/nodes/task/pre_post_execution.go @@ -2,26 +2,110 @@ package task import ( "context" - "time" + //"time" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" - pluginCore "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/core" + //pluginCore "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" + "github.com/flyteorg/flytestdlib/contextutils" "github.com/flyteorg/flytestdlib/logger" - "github.com/pkg/errors" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" + //"github.com/pkg/errors" + //"google.golang.org/grpc/codes" + //"google.golang.org/grpc/status" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" errors2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" ) var cacheDisabled = catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil) -func (t *Handler) CheckCatalogCache(ctx context.Context, tr pluginCore.TaskReader, inputReader io.InputReader, outputWriter io.OutputWriter) (catalog.Entry, error) { +func (t *Handler) GetCatalogKey(ctx context.Context, nCtx handler.NodeExecutionContext) (catalog.Key, error) { + // read task template + taskTemplatePath, err := ioutils.GetTaskTemplatePath(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetDataDir()) + if err != nil { + return catalog.Key{}, err + } + + taskReader := ioutils.NewLazyUploadingTaskReader(nCtx.TaskReader(), taskTemplatePath, nCtx.DataStore()) + taskTemplate, err := taskReader.Read(ctx) + if err != nil { + logger.Errorf(ctx, "failed to read TaskTemplate, error :%s", err.Error()) + return catalog.Key{}, err + } + + return catalog.Key{ + Identifier: *taskTemplate.Id, + CacheVersion: taskTemplate.Metadata.DiscoveryVersion, + TypedInterface: *taskTemplate.Interface, + InputReader: nCtx.InputReader(), + }, nil +} + +func (t *Handler) IsCacheable(ctx context.Context, nCtx handler.NodeExecutionContext) (bool, error) { + // check if plugin has caching disabled + ttype := nCtx.TaskReader().GetTaskType() + ctx = contextutils.WithTaskType(ctx, ttype) + p, err := t.ResolvePlugin(ctx, ttype, nCtx.ExecutionContext().GetExecutionConfig()) + if err != nil { + return false, errors2.Wrapf(errors2.UnsupportedTaskTypeError, nCtx.NodeID(), err, "unable to resolve plugin") + } + + checkCatalog := !p.GetProperties().DisableNodeLevelCaching + if !checkCatalog { + logger.Infof(ctx, "Node level caching is disabled. Skipping catalog read.") + } + + // read task template + taskTemplatePath, err := ioutils.GetTaskTemplatePath(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetDataDir()) + if err != nil { + return false, err + } + + taskReader := ioutils.NewLazyUploadingTaskReader(nCtx.TaskReader(), taskTemplatePath, nCtx.DataStore()) + taskTemplate, err := taskReader.Read(ctx) + if err != nil { + logger.Errorf(ctx, "failed to read TaskTemplate, error :%s", err.Error()) + return false, err + } + + return taskTemplate.Metadata.Discoverable, nil +} + +func (t *Handler) IsCacheSerializable(ctx context.Context, nCtx handler.NodeExecutionContext) (bool, error) { + // check if plugin has caching disabled + ttype := nCtx.TaskReader().GetTaskType() + ctx = contextutils.WithTaskType(ctx, ttype) + p, err := t.ResolvePlugin(ctx, ttype, nCtx.ExecutionContext().GetExecutionConfig()) + if err != nil { + return false, errors2.Wrapf(errors2.UnsupportedTaskTypeError, nCtx.NodeID(), err, "unable to resolve plugin") + } + + checkCatalog := !p.GetProperties().DisableNodeLevelCaching + if !checkCatalog { + logger.Infof(ctx, "Node level caching is disabled. Skipping catalog read.") + } + + // read task template + taskTemplatePath, err := ioutils.GetTaskTemplatePath(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetDataDir()) + if err != nil { + return false, err + } + + taskReader := ioutils.NewLazyUploadingTaskReader(nCtx.TaskReader(), taskTemplatePath, nCtx.DataStore()) + taskTemplate, err := taskReader.Read(ctx) + if err != nil { + logger.Errorf(ctx, "failed to read TaskTemplate, error :%s", err.Error()) + return false, err + } + + return taskTemplate.Metadata.Discoverable && taskTemplate.Metadata.CacheSerializable, nil +} + +/*func (t *Handler) CheckCatalogCache(ctx context.Context, tr pluginCore.TaskReader, inputReader io.InputReader, outputWriter io.OutputWriter) (catalog.Entry, error) { tk, err := tr.Read(ctx) if err != nil { logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) @@ -113,7 +197,7 @@ func (t *Handler) GetOrExtendCatalogReservation(ctx context.Context, ownerID str } logger.Infof(ctx, "Catalog CacheSerializeDisabled: for Task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), nil -} +}*/ func (t *Handler) ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, @@ -224,16 +308,16 @@ func (t *Handler) ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1 // ignores discovery write failures s, err2 := t.catalog.Put(ctx, key, r, m) if err2 != nil { - t.metrics.catalogPutFailureCount.Inc(ctx) + //t.metrics.catalogPutFailureCount.Inc(ctx) logger.Errorf(ctx, "Failed to write results to catalog for Task [%v]. Error: %v", tk.GetId(), err2) return catalog.NewStatus(core.CatalogCacheStatus_CACHE_PUT_FAILURE, s.GetMetadata()), nil, nil } - t.metrics.catalogPutSuccessCount.Inc(ctx) + //t.metrics.catalogPutSuccessCount.Inc(ctx) logger.Infof(ctx, "Successfully cached results to catalog - Task [%v]", tk.GetId()) return s, nil, nil } -// ReleaseCatalogReservation attempts to release an artifact reservation if the task is cachable +/*// ReleaseCatalogReservation attempts to release an artifact reservation if the task is cachable // and cache serializable. If the reservation does not exist for this owner (e.x. it never existed // or has been acquired by another owner) this call is still successful. func (t *Handler) ReleaseCatalogReservation(ctx context.Context, ownerID string, tr pluginCore.TaskReader, inputReader io.InputReader) (catalog.ReservationEntry, error) { @@ -264,4 +348,4 @@ func (t *Handler) ReleaseCatalogReservation(ctx context.Context, ownerID string, } logger.Infof(ctx, "Catalog CacheSerializeDisabled: for Task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), nil -} +}*/ From 16d3fbc3d73b73e24e7ca0af5244c7eb221e173a Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 26 Sep 2022 11:40:21 -0500 Subject: [PATCH 02/40] processing downstream immediately on cache hit Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 29869808a..b2d76affd 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -474,7 +474,7 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusPending, nil } - // TODO @hamersaw + // TODO @hamersaw - complete if cacheHandler, ok := h.(handler.CacheableNode); ok { cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { @@ -519,7 +519,7 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor nodeStatus.ClearSubNodeStatus() nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) - // TODO send events? (with cache hit) - for some reason this is not showing up as CACHE_HIT in UI + // send execution events - TODO @hamersaw UI is not showing CACHE_HIT, do we need a TaskExecutionEvent? phaseInfo := handler.PhaseInfoSuccess(&handler.ExecutionInfo{ OutputInfo: &handler.OutputInfo { OutputURI: outputFile, @@ -544,9 +544,8 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusUndefined, errors.Wrapf(errors.EventRecordingFailed, nCtx.NodeID(), err, "failed to record node event") } - // TODO return handleDownstream() - return executors.NodeStatusSuccess, nil - //return c.handleDownstream(ctx, execContext, dag, nl, currentNode) + // process downstream nodes + return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } } } From 08eda3174079329a8af62146addeb0f50378fc82 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 6 Oct 2022 16:05:51 -0500 Subject: [PATCH 03/40] moved cache write to node executor Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 14 ++++ pkg/controller/nodes/dynamic/handler.go | 22 +++-- pkg/controller/nodes/executor.go | 51 ++++++++++++ pkg/controller/nodes/task/handler.go | 8 +- .../nodes/task/pre_post_execution.go | 80 ++++++++++++++++++- 5 files changed, 163 insertions(+), 12 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index f63dcc612..7e3e36648 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -6,6 +6,7 @@ import ( "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" "github.com/flyteorg/flytestdlib/logger" @@ -64,3 +65,16 @@ func (n *nodeExecutor) CheckCacheCatalog(ctx context.Context, key catalog.Key) ( // SetCached. return resp, nil } + +func (n *nodeExecutor) WriteCacheCatalog(ctx context.Context, key catalog.Key, outputReader io.OutputReader, metadata catalog.Metadata) (catalog.Status, error) { + //logger.Infof(ctx, "Catalog CacheEnabled. recording execution [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) + // ignores discovery write failures + status, err := n.catalog.Put(ctx, key, outputReader, metadata) + if err != nil { + //t.metrics.catalogPutFailureCount.Inc(ctx) + //logger.Errorf(ctx, "Failed to write results to catalog for Task [%v]. Error: %v", tk.GetId(), err2) + return catalog.NewStatus(core.CatalogCacheStatus_CACHE_PUT_FAILURE, status.GetMetadata()), nil + } + + return status, nil +} diff --git a/pkg/controller/nodes/dynamic/handler.go b/pkg/controller/nodes/dynamic/handler.go index d1fd89ba2..2a84c279b 100644 --- a/pkg/controller/nodes/dynamic/handler.go +++ b/pkg/controller/nodes/dynamic/handler.go @@ -7,7 +7,7 @@ import ( "github.com/flyteorg/flytepropeller/pkg/controller/config" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + //"github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flytestdlib/logger" @@ -34,9 +34,12 @@ const dynamicNodeID = "dynamic-node" type TaskNodeHandler interface { handler.CacheableNode - ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, + /*ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, - tr ioutils.SimpleTaskReader, m catalog.Metadata) (catalog.Status, *io.ExecutionError, error) + tr ioutils.SimpleTaskReader, m catalog.Metadata) (catalog.Status, *io.ExecutionError, error)*/ + ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, + r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, + tr ioutils.SimpleTaskReader) (*io.ExecutionError, error) } type metrics struct { @@ -146,12 +149,15 @@ func (d dynamicNodeTaskNodeHandler) handleDynamicSubNodes(ctx context.Context, n // These outputPaths only reads the output metadata. So the sandbox is completely optional here and hence it is nil. // The sandbox creation as it uses hashing can be expensive and we skip that expense. outputPaths := ioutils.NewReadOnlyOutputFilePaths(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetOutputDir()) - execID := task.GetTaskExecutionIdentifier(nCtx) + //execID := task.GetTaskExecutionIdentifier(nCtx) outputReader := ioutils.NewRemoteFileOutputReader(ctx, nCtx.DataStore(), outputPaths, nCtx.MaxDatasetSizeBytes()) - status, ee, err := d.TaskNodeHandler.ValidateOutputAndCacheAdd(ctx, nCtx.NodeID(), nCtx.InputReader(), + /*status, ee, err := d.TaskNodeHandler.ValidateOutputAndCacheAdd(ctx, nCtx.NodeID(), nCtx.InputReader(), outputReader, nil, nCtx.ExecutionContext().GetExecutionConfig(), nCtx.TaskReader(), catalog.Metadata{ TaskExecutionIdentifier: execID, - }) + })*/ + + ee, err := d.TaskNodeHandler.ValidateOutput(ctx, nCtx.NodeID(), nCtx.InputReader(), + outputReader, nil, nCtx.ExecutionContext().GetExecutionConfig(), nCtx.TaskReader()) if err != nil { return handler.UnknownTransition, prevState, err @@ -164,7 +170,9 @@ func (d dynamicNodeTaskNodeHandler) handleDynamicSubNodes(ctx context.Context, n return trns.WithInfo(handler.PhaseInfoFailureErr(ee.ExecutionError, trns.Info().GetInfo())), handler.DynamicNodeState{Phase: v1alpha1.DynamicNodePhaseFailing, Reason: ee.ExecutionError.String()}, nil } - taskNodeInfoMetadata := &event.TaskNodeMetadata{CacheStatus: status.GetCacheStatus(), CatalogKey: status.GetMetadata()} + // TODO @hamersaw - do we need to even populate this? + //taskNodeInfoMetadata := &event.TaskNodeMetadata{CacheStatus: status.GetCacheStatus(), CatalogKey: status.GetMetadata()} + taskNodeInfoMetadata := &event.TaskNodeMetadata{} trns.WithInfo(trns.Info().WithInfo(&handler.ExecutionInfo{TaskNodeInfo: &handler.TaskNodeInfo{TaskNodeMetadata: taskNodeInfoMetadata}})) } diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 76b799bac..e00a9bb47 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -54,6 +54,7 @@ import ( "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task" ) type nodeMetrics struct { @@ -447,6 +448,56 @@ func (c *nodeExecutor) execute(ctx context.Context, h handler.Node, nCtx *nodeEx nCtx.nsm.clearNodeStatus() } + // TODO @hamersaw - complete + if phase.GetPhase() == handler.EPhaseSuccess { + if cacheHandler, ok := h.(handler.CacheableNode); ok { + cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return handler.PhaseInfoUndefined, err + } + + if cacheable { + cacheCatalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) + if err != nil { + // TODO @hamersaw fail + return handler.PhaseInfoUndefined, err + } + + outputPaths := ioutils.NewReadOnlyOutputFilePaths(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetOutputDir()) + outputReader := ioutils.NewRemoteFileOutputReader(ctx, nCtx.DataStore(), outputPaths, nCtx.MaxDatasetSizeBytes()) + + // TODO @hamersaw - need to update this once we support caching of non-tasks + metadata := catalog.Metadata{ + TaskExecutionIdentifier: task.GetTaskExecutionIdentifier(nCtx), + } + + status, err := c.WriteCacheCatalog(ctx, cacheCatalogKey, outputReader, metadata) + if err != nil { + // TODO @hamersaw fail + return handler.PhaseInfoUndefined, err + } + + // populate PhaseInfo with cache status + info := phase.GetInfo() + if info == nil { + info = &handler.ExecutionInfo{} + } + + if info.TaskNodeInfo == nil { + info.TaskNodeInfo = &handler.TaskNodeInfo{} + } + if info.TaskNodeInfo.TaskNodeMetadata == nil { + info.TaskNodeInfo.TaskNodeMetadata = &event.TaskNodeMetadata{} + } + + info.TaskNodeInfo.TaskNodeMetadata.CacheStatus = status.GetCacheStatus() + info.TaskNodeInfo.TaskNodeMetadata.CatalogKey = status.GetMetadata() + phase = phase.WithInfo(info) + } + } + } + return phase, nil } diff --git a/pkg/controller/nodes/task/handler.go b/pkg/controller/nodes/task/handler.go index e5671f480..eb24a6f65 100644 --- a/pkg/controller/nodes/task/handler.go +++ b/pkg/controller/nodes/task/handler.go @@ -487,11 +487,13 @@ func (t Handler) invokePlugin(ctx context.Context, p pluginCore.Plugin, tCtx *ta // ------------------------------------- logger.Debugf(ctx, "Task success detected, calling on Task success") outputCommitter := ioutils.NewRemoteFileOutputWriter(ctx, tCtx.DataStore(), tCtx.OutputWriter()) - execID := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetID() + /*execID := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetID() cacheStatus, ee, err := t.ValidateOutputAndCacheAdd(ctx, tCtx.NodeID(), tCtx.InputReader(), tCtx.ow.GetReader(), outputCommitter, tCtx.ExecutionContext().GetExecutionConfig(), tCtx.tr, catalog.Metadata{ TaskExecutionIdentifier: &execID, - }) + })*/ + ee, err := t.ValidateOutput(ctx, tCtx.NodeID(), tCtx.InputReader(), tCtx.ow.GetReader(), + outputCommitter, tCtx.ExecutionContext().GetExecutionConfig(), tCtx.tr) if err != nil { return nil, err } @@ -515,8 +517,6 @@ func (t Handler) invokePlugin(ctx context.Context, p pluginCore.Plugin, tCtx *ta } pluginTrns.ObserveSuccess(tCtx.ow.GetOutputPath(), deckURI, &event.TaskNodeMetadata{ - CacheStatus: cacheStatus.GetCacheStatus(), - CatalogKey: cacheStatus.GetMetadata(), CheckpointUri: tCtx.ow.GetCheckpointPrefix().String(), }) } diff --git a/pkg/controller/nodes/task/pre_post_execution.go b/pkg/controller/nodes/task/pre_post_execution.go index 73f6ae85a..31af864a2 100644 --- a/pkg/controller/nodes/task/pre_post_execution.go +++ b/pkg/controller/nodes/task/pre_post_execution.go @@ -199,7 +199,7 @@ func (t *Handler) GetOrExtendCatalogReservation(ctx context.Context, ownerID str return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), nil }*/ -func (t *Handler) ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, +/*func (t *Handler) ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader, m catalog.Metadata) (catalog.Status, *io.ExecutionError, error) { @@ -311,6 +311,84 @@ func (t *Handler) ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1 //t.metrics.catalogPutSuccessCount.Inc(ctx) logger.Infof(ctx, "Successfully cached results to catalog - Task [%v]", tk.GetId()) return s, nil, nil +}*/ + +func (t *Handler) ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, + r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, + tr ioutils.SimpleTaskReader) (*io.ExecutionError, error) { + + tk, err := tr.Read(ctx) + if err != nil { + logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) + return nil, err + } + + iface := tk.Interface + outputsDeclared := iface != nil && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 + + if r == nil { + if outputsDeclared { + // Whack! plugin did not return any outputs for this task + // Also When an error is observed, cache is automatically disabled + return &io.ExecutionError{ + ExecutionError: &core.ExecutionError{ + Code: "OutputsNotGenerated", + Message: "Output Reader was nil. Plugin/Platform problem.", + }, + IsRecoverable: true, + }, nil + } + return nil, nil + } + // Reader exists, we can check for error, even if this task may not have any outputs declared + y, err := r.IsError(ctx) + if err != nil { + return nil, err + } + if y { + taskErr, err := r.ReadError(ctx) + if err != nil { + return nil, err + } + + if taskErr.ExecutionError == nil { + taskErr.ExecutionError = &core.ExecutionError{Kind: core.ExecutionError_UNKNOWN, Code: "Unknown", Message: "Unknown"} + } + return &taskErr, nil + } + + // Do this if we have outputs declared for the Handler interface! + if !outputsDeclared { + return nil, nil + } + ok, err := r.Exists(ctx) + if err != nil { + logger.Errorf(ctx, "Failed to check if the output file exists. Error: %s", err.Error()) + return nil, err + } + + if !ok { + // Does not exist + return &io.ExecutionError{ + ExecutionError: &core.ExecutionError{ + Code: "OutputsNotFound", + Message: "Outputs not generated by task execution", + }, + IsRecoverable: true, + }, nil + } + + if !r.IsFile(ctx) { + // Read output and write to file + // No need to check for Execution Error here as we have done so above this block. + err = outputCommitter.Put(ctx, r) + if err != nil { + logger.Errorf(ctx, "Failed to commit output to remote location. Error: %v", err) + return nil, err + } + } + + return nil, nil } /*// ReleaseCatalogReservation attempts to release an artifact reservation if the task is cachable From ac773ae037a6b534817d9f26c35ac954fedc1f67 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sat, 8 Oct 2022 05:50:17 -0500 Subject: [PATCH 04/40] working cache and cache serialize Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 197 ++++++++++++++++++++--- pkg/controller/nodes/dynamic/handler.go | 4 +- pkg/controller/nodes/executor.go | 200 +++++++++++++++--------- 3 files changed, 308 insertions(+), 93 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 7e3e36648..ddafc6b66 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -2,13 +2,21 @@ package nodes import ( "context" + "time" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" + + "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/common" + nodeserrors "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task" "github.com/flyteorg/flytestdlib/logger" + "github.com/flyteorg/flytestdlib/storage" "github.com/pkg/errors" @@ -31,9 +39,15 @@ import ( errors2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors"*/ ) -func (n *nodeExecutor) CheckCacheCatalog(ctx context.Context, key catalog.Key) (catalog.Entry, error) { +func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { + catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) + if err != nil { + // TODO @hamersaw fail + return catalog.Entry{}, err + } + //logger.Infof(ctx, "Catalog CacheEnabled: Looking up catalog Cache.") - resp, err := n.catalog.Get(ctx, key) + entry, err := n.catalog.Get(ctx, catalogKey) if err != nil { causeErr := errors.Cause(err) if taskStatus, ok := status.FromError(causeErr); ok && taskStatus.Code() == codes.NotFound { @@ -47,29 +61,174 @@ func (n *nodeExecutor) CheckCacheCatalog(ctx context.Context, key catalog.Key) ( return catalog.Entry{}, errors.Wrapf(err, "Failed to check Catalog for previous results") } - if resp.GetStatus().GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT { - logger.Errorf(ctx, "No CacheHIT and no Error received. Illegal state, Cache State: %s", resp.GetStatus().GetCacheStatus().String()) - // TODO should this be an error? - return resp, nil - } + // TODO @hamersaw - figure out + iface := catalogKey.TypedInterface + if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 { + // copy cached outputs to node outputs + o, ee, err := entry.GetOutputs().Read(ctx) + if err != nil { + logger.Errorf(ctx, "failed to read from catalog, err: %s", err.Error()) + return catalog.Entry{}, err + } else if ee != nil { + logger.Errorf(ctx, "got execution error from catalog output reader? This should not happen, err: %s", ee.String()) + return catalog.Entry{}, nodeserrors.Errorf(nodeserrors.IllegalStateError, nCtx.NodeID(), "execution error from a cache output, bad state: %s", ee.String()) + } - // TODO @hamersaw - do we need this?!?! - //logger.Infof(ctx, "Catalog CacheHit: for task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) - //t.metrics.catalogHitCount.Inc(ctx) - /*if iface := tk.Interface; iface != nil && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 { - if err := outputWriter.Put(ctx, resp.GetOutputs()); err != nil { - logger.Errorf(ctx, "failed to write data to Storage, err: %v", err.Error()) - return catalog.Entry{}, errors.Wrapf(err, "failed to copy cached results for task.") + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + if err := nCtx.DataStore().WriteProtobuf(ctx, outputFile, storage.Options{}, o); err != nil { + logger.Errorf(ctx, "failed to write cached value to datastore, err: %s", err.Error()) + return catalog.Entry{}, err } - }*/ + } + // SetCached. - return resp, nil + //logger.Errorf(ctx, "No CacheHIT and no Error received. Illegal state, Cache State: %s", entry.GetStatus().GetCacheStatus().String()) + // TODO should this be an error? + return entry, nil +} + +// GetOrExtendCatalogReservation attempts to acquire an artifact reservation if the task is +// cachable and cache serializable. If the reservation already exists for this owner, the +// reservation is extended. +func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx *nodeExecContext, + cacheHandler handler.CacheableNode, heartbeatInterval time.Duration) (catalog.ReservationEntry, error) { + + catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) + if err != nil { + // TODO @hamersaw fail + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + } + + // compute ownerID + currentNodeUniqueID, err := common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) + if err != nil { + // TODO @hamersaw - fail + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + } + + _, ownerID, err := task.ComputeRawOutputPrefix(ctx, task.IDMaxLength, nCtx, currentNodeUniqueID, nCtx.CurrentAttempt()) + if err != nil { + // TODO @hamersaw - fail + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + } + + reservation, err := n.catalog.GetOrExtendReservation(ctx, catalogKey, ownerID, heartbeatInterval) + if err != nil { + //t.metrics.reservationGetFailureCount.Inc(ctx) + logger.Errorf(ctx, "Catalog Failure: reservation get or extend failed. err: %v", err.Error()) + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err + } + + var status core.CatalogReservation_Status + if reservation.OwnerId == ownerID { + status = core.CatalogReservation_RESERVATION_ACQUIRED + } else { + status = core.CatalogReservation_RESERVATION_EXISTS + } + + //t.metrics.reservationGetSuccessCount.Inc(ctx) + return catalog.NewReservationEntry(reservation.ExpiresAt.AsTime(), + reservation.HeartbeatInterval.AsDuration(), reservation.OwnerId, status), nil + + /*tk, err := tr.Read(ctx) + if err != nil { + logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err + } + + if tk.Metadata.Discoverable && tk.Metadata.CacheSerializable { + logger.Infof(ctx, "Catalog CacheSerializeEnabled: creating catalog reservation.") + key := catalog.Key{ + Identifier: *tk.Id, + CacheVersion: tk.Metadata.DiscoveryVersion, + TypedInterface: *tk.Interface, + InputReader: inputReader, + } + + reservation, err := t.catalog.GetOrExtendReservation(ctx, key, ownerID, heartbeatInterval) + if err != nil { + t.metrics.reservationGetFailureCount.Inc(ctx) + logger.Errorf(ctx, "Catalog Failure: reservation get or extend failed. err: %v", err.Error()) + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err + } + + expiresAt := reservation.ExpiresAt.AsTime() + heartbeatInterval := reservation.HeartbeatInterval.AsDuration() + + var status core.CatalogReservation_Status + if reservation.OwnerId == ownerID { + status = core.CatalogReservation_RESERVATION_ACQUIRED + } else { + status = core.CatalogReservation_RESERVATION_EXISTS + } + + t.metrics.reservationGetSuccessCount.Inc(ctx) + return catalog.NewReservationEntry(expiresAt, heartbeatInterval, reservation.OwnerId, status), nil + } + logger.Infof(ctx, "Catalog CacheSerializeDisabled: for Task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), nil*/ +} + +// ReleaseCatalogReservation attempts to release an artifact reservation if the task is cachable +// and cache serializable. If the reservation does not exist for this owner (e.x. it never existed +// or has been acquired by another owner) this call is still successful. +func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *nodeExecContext, + cacheHandler handler.CacheableNode) (catalog.ReservationEntry, error) { + + catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) + if err != nil { + // TODO @hamersaw fail + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + } + + // compute ownerID - TODO @hamersaw make separate function (duplicated code) + currentNodeUniqueID, err := common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) + if err != nil { + // TODO @hamersaw - fail + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + } + + _, ownerID, err := task.ComputeRawOutputPrefix(ctx, task.IDMaxLength, nCtx, currentNodeUniqueID, nCtx.CurrentAttempt()) + if err != nil { + // TODO @hamersaw - fail + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + } + + err = n.catalog.ReleaseReservation(ctx, catalogKey, ownerID) + if err != nil { + //t.metrics.reservationReleaseFailureCount.Inc(ctx) + logger.Errorf(ctx, "Catalog Failure: release reservation failed. err: %v", err.Error()) + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err + } + + //t.metrics.reservationReleaseSuccessCount.Inc(ctx) + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_RELEASED), nil } -func (n *nodeExecutor) WriteCacheCatalog(ctx context.Context, key catalog.Key, outputReader io.OutputReader, metadata catalog.Metadata) (catalog.Status, error) { +func (n *nodeExecutor) WriteCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Status, error) { + catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) + if err != nil { + // TODO @hamersaw fail + return catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), err + } + + // TODO @hamersaw - is this right? + /*iface := catalogKey.TypedInterface + if iface.Outputs != nil && len(iface.Outputs.Variables) == 0 { + return catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), nil + }*/ + + outputPaths := ioutils.NewReadOnlyOutputFilePaths(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetOutputDir()) + outputReader := ioutils.NewRemoteFileOutputReader(ctx, nCtx.DataStore(), outputPaths, nCtx.MaxDatasetSizeBytes()) + + // TODO @hamersaw - need to update this once we support caching of non-tasks + metadata := catalog.Metadata{ + TaskExecutionIdentifier: task.GetTaskExecutionIdentifier(nCtx), + } + //logger.Infof(ctx, "Catalog CacheEnabled. recording execution [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) // ignores discovery write failures - status, err := n.catalog.Put(ctx, key, outputReader, metadata) + status, err := n.catalog.Put(ctx, catalogKey, outputReader, metadata) if err != nil { //t.metrics.catalogPutFailureCount.Inc(ctx) //logger.Errorf(ctx, "Failed to write results to catalog for Task [%v]. Error: %v", tk.GetId(), err2) diff --git a/pkg/controller/nodes/dynamic/handler.go b/pkg/controller/nodes/dynamic/handler.go index 2a84c279b..05b599483 100644 --- a/pkg/controller/nodes/dynamic/handler.go +++ b/pkg/controller/nodes/dynamic/handler.go @@ -172,8 +172,8 @@ func (d dynamicNodeTaskNodeHandler) handleDynamicSubNodes(ctx context.Context, n } // TODO @hamersaw - do we need to even populate this? //taskNodeInfoMetadata := &event.TaskNodeMetadata{CacheStatus: status.GetCacheStatus(), CatalogKey: status.GetMetadata()} - taskNodeInfoMetadata := &event.TaskNodeMetadata{} - trns.WithInfo(trns.Info().WithInfo(&handler.ExecutionInfo{TaskNodeInfo: &handler.TaskNodeInfo{TaskNodeMetadata: taskNodeInfoMetadata}})) + //trns.WithInfo(trns.Info().WithInfo(&handler.ExecutionInfo{TaskNodeInfo: &handler.TaskNodeInfo{TaskNodeMetadata: taskNodeInfoMetadata}})) + trns.WithInfo(trns.Info().WithInfo(&handler.ExecutionInfo{TaskNodeInfo: &handler.TaskNodeInfo{TaskNodeMetadata: &event.TaskNodeMetadata{}}})) } return trns, newState, nil diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index e00a9bb47..8dc8b2e38 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -54,7 +54,6 @@ import ( "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task" ) type nodeMetrics struct { @@ -455,29 +454,14 @@ func (c *nodeExecutor) execute(ctx context.Context, h handler.Node, nCtx *nodeEx if err != nil { logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) return handler.PhaseInfoUndefined, err - } - - if cacheable { - cacheCatalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) - if err != nil { - // TODO @hamersaw fail - return handler.PhaseInfoUndefined, err - } - - outputPaths := ioutils.NewReadOnlyOutputFilePaths(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetOutputDir()) - outputReader := ioutils.NewRemoteFileOutputReader(ctx, nCtx.DataStore(), outputPaths, nCtx.MaxDatasetSizeBytes()) - - // TODO @hamersaw - need to update this once we support caching of non-tasks - metadata := catalog.Metadata{ - TaskExecutionIdentifier: task.GetTaskExecutionIdentifier(nCtx), - } - - status, err := c.WriteCacheCatalog(ctx, cacheCatalogKey, outputReader, metadata) + } else if cacheable { + status, err := c.WriteCatalogCache(ctx, nCtx, cacheHandler) if err != nil { // TODO @hamersaw fail return handler.PhaseInfoUndefined, err } + // TODO @hamersaw - does this need to be outside? // populate PhaseInfo with cache status info := phase.GetInfo() if info == nil { @@ -487,6 +471,7 @@ func (c *nodeExecutor) execute(ctx context.Context, h handler.Node, nCtx *nodeEx if info.TaskNodeInfo == nil { info.TaskNodeInfo = &handler.TaskNodeInfo{} } + if info.TaskNodeInfo.TaskNodeMetadata == nil { info.TaskNodeInfo.TaskNodeMetadata = &event.TaskNodeMetadata{} } @@ -495,6 +480,18 @@ func (c *nodeExecutor) execute(ctx context.Context, h handler.Node, nCtx *nodeEx info.TaskNodeInfo.TaskNodeMetadata.CatalogKey = status.GetMetadata() phase = phase.WithInfo(info) } + + cacheSerializable, err := cacheHandler.IsCacheSerializable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return handler.PhaseInfoUndefined, err + } else if cacheSerializable { + _, err := c.ReleaseCatalogReservation(ctx, nCtx, cacheHandler) + if err != nil { + // TODO @hamersaw fail + return handler.PhaseInfoUndefined, err + } + } } } @@ -538,52 +535,28 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor } // TODO @hamersaw - complete + cacheStatus := core.CatalogCacheStatus_CACHE_DISABLED if cacheHandler, ok := h.(handler.CacheableNode); ok { cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) return executors.NodeStatusUndefined, err - } - - if cacheable { - cacheCatalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) + } else if cacheable { + entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) if err != nil { // TODO @hamersaw fail return executors.NodeStatusUndefined, err } - entry, err := c.CheckCacheCatalog(ctx, cacheCatalogKey) - if err != nil { - // TODO @hamersaw fail - return executors.NodeStatusUndefined, err - } - - // TODO @hamersaw - figure out - if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - // copy cached outputs to node outputs - o, ee, err := entry.GetOutputs().Read(ctx) - if err != nil { - logger.Errorf(ctx, "failed to read from catalog, err: %s", err.Error()) - return executors.NodeStatusUndefined, err - } - - if ee != nil { - logger.Errorf(ctx, "got execution error from catalog output reader? This should not happen, err: %s", ee.String()) - return executors.NodeStatusUndefined, errors.Errorf(errors.IllegalStateError, nCtx.NodeID(), "execution error from a cache output, bad state: %s", ee.String()) - } - - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) - if err := nCtx.DataStore().WriteProtobuf(ctx, outputFile, storage.Options{}, o); err != nil { - logger.Errorf(ctx, "failed to write cached value to datastore, err: %s", err.Error()) - return executors.NodeStatusUndefined, err - } - + cacheStatus = entry.GetStatus().GetCacheStatus() + if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { // update NodeStatus to Success nodeStatus.ClearSubNodeStatus() nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) - // send execution events - TODO @hamersaw UI is not showing CACHE_HIT, do we need a TaskExecutionEvent? - phaseInfo := handler.PhaseInfoSuccess(&handler.ExecutionInfo{ + // set phaseInfo transition to include ... TODO @hamersaw + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ OutputInfo: &handler.OutputInfo { OutputURI: outputFile, }, @@ -594,21 +567,6 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor }, }, }) - - nev, err := ToNodeExecutionEvent(nCtx.NodeExecutionMetadata().GetNodeExecutionID(), - phaseInfo, nCtx.InputReader().GetInputPath().String(), nodeStatus, nCtx.ExecutionContext().GetEventVersion(), - nCtx.ExecutionContext().GetParentInfo(), nCtx.node, c.clusterID, nCtx.NodeStateReader().GetDynamicNodeState().Phase) - if err != nil { - return executors.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "could not convert phase info to event") - } - err = c.IdempotentRecordEvent(ctx, nev) - if err != nil { - logger.Warningf(ctx, "Failed to record nodeEvent, error [%s]", err.Error()) - return executors.NodeStatusUndefined, errors.Wrapf(errors.EventRecordingFailed, nCtx.NodeID(), err, "failed to record node event") - } - - // process downstream nodes - return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } } } @@ -644,27 +602,118 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusQueued, nil } else if np == v1alpha1.NodePhaseSkipped { return executors.NodeStatusSuccess, nil + } else if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { + // process downstream nodes + return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } return executors.NodeStatusPending, nil } -func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, nCtx *nodeExecContext, h handler.Node) (executors.NodeStatus, error) { +func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag executors.DAGStructure, nCtx *nodeExecContext, h handler.Node) (executors.NodeStatus, error) { nodeStatus := nCtx.NodeStatus() currentPhase := nodeStatus.GetPhase() + p := handler.PhaseInfoUndefined // case v1alpha1.NodePhaseQueued, v1alpha1.NodePhaseRunning: logger.Debugf(ctx, "node executing, current phase [%s]", currentPhase) defer logger.Debugf(ctx, "node execution completed") + // TODO @hamersaw - check cache serialize + cacheStatus := core.CatalogCacheStatus_CACHE_DISABLED + catalogReservationStatus := core.CatalogReservation_RESERVATION_DISABLED + if cacheHandler, ok := h.(handler.CacheableNode); ok { + // TODO @hamersaw - we are checking cache twice when nodes start ... need to fix this! + if currentPhase == v1alpha1.NodePhaseQueued { + cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return executors.NodeStatusUndefined, err + } else if cacheable { + entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) + if err != nil { + // TODO @hamersaw fail + return executors.NodeStatusUndefined, err + } + + cacheStatus = entry.GetStatus().GetCacheStatus() + if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { + // update NodeStatus to Success + nodeStatus.ClearSubNodeStatus() + nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) + + // set phaseInfo transition to include ... TODO @hamersaw + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ + OutputInfo: &handler.OutputInfo { + OutputURI: outputFile, + }, + TaskNodeInfo: &handler.TaskNodeInfo { + TaskNodeMetadata: &event.TaskNodeMetadata{ + CacheStatus: entry.GetStatus().GetCacheStatus(), + CatalogKey: entry.GetStatus().GetMetadata(), + }, + }, + }) + } + } + } + + if cacheStatus != core.CatalogCacheStatus_CACHE_HIT { + cacheSerializable, err := cacheHandler.IsCacheSerializable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return executors.NodeStatusUndefined, err + } else if cacheSerializable { + entry, err := c.GetOrExtendCatalogReservation(ctx, nCtx, cacheHandler, config.GetConfig().WorkflowReEval.Duration) + if err != nil { + // TODO @hamersaw fail + return executors.NodeStatusUndefined, err + } + + // TODO @hamersaw - update phase info with reservation info + //p.execInfo.TaskNodeInfo.TaskNodeMetadata.ReservationStatus = entry.GetStatus() + //pluginTrns.PopulateReservationInfo(reservation) + + /*if reservation.GetStatus() == core.CatalogReservation_RESERVATION_ACQUIRED && + (ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache) { + logger.Infof(ctx, "Acquired cache reservation") + }*/ + + // If we do not own the reservation then we transition to WaitingForCache phase. If we are + // already running (ie. in a phase other than PhaseUndefined or PhaseWaitingForCache) and + // somehow lost the reservation (ex. by expiration), continue to execute until completion. + catalogReservationStatus = entry.GetStatus() + if catalogReservationStatus == core.CatalogReservation_RESERVATION_EXISTS { + /*if ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache { + pluginTrns.ttype = handler.TransitionTypeEphemeral + pluginTrns.pInfo = pluginCore.PhaseInfoWaitingForCache(pluginCore.DefaultPhaseVersion, nil) + }*/ + + if currentPhase == v1alpha1.NodePhaseQueued { + //if ts.PluginPhase == pluginCore.PhaseWaitingForCache { + logger.Debugf(ctx, "No state change for Task, previously observed same transition. Short circuiting.") + //return executors.NodeStatusQueued, nil // TODO @hamersaw - should probably fallthrough on this + p = handler.PhaseInfoQueued("TODO hamersaw") + } + } + } + } + } + // Since we reset node status inside execute for retryable failure, we use lastAttemptStartTime to carry that information // across execute which is used to emit metrics lastAttemptStartTime := nodeStatus.GetLastAttemptStartedAt() - p, err := c.execute(ctx, h, nCtx, nodeStatus) - if err != nil { - logger.Errorf(ctx, "failed Execute for node. Error: %s", err.Error()) - return executors.NodeStatusUndefined, err + // TODO @hamersaw - document + if currentPhase != v1alpha1.NodePhaseQueued || (cacheStatus != core.CatalogCacheStatus_CACHE_HIT && catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS) { + var err error + + p, err = c.execute(ctx, h, nCtx, nodeStatus) + if err != nil { + logger.Errorf(ctx, "failed Execute for node. Error: %s", err.Error()) + return executors.NodeStatusUndefined, err + } } if p.GetPhase() == handler.EPhaseUndefined { @@ -726,6 +775,7 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, nCtx *node np = v1alpha1.NodePhaseSucceeded finalStatus = executors.NodeStatusSuccess } + if np == v1alpha1.NodePhaseRecovered { logger.Infof(ctx, "Finalize not required, moving node to Recovered") finalStatus = executors.NodeStatusRecovered @@ -783,6 +833,12 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, nCtx *node } UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) + + if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { + // process downstream nodes + return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) + } + return finalStatus, nil } @@ -886,7 +942,7 @@ func (c *nodeExecutor) handleNode(ctx context.Context, dag executors.DAGStructur return executors.NodeStatusFailed(nodeStatus.GetExecutionError()), nil } - return c.handleQueuedOrRunningNode(ctx, nCtx, h) + return c.handleQueuedOrRunningNode(ctx, dag, nCtx, h) } // The space search for the next node to execute is implemented like a DFS algorithm. handleDownstream visits all the nodes downstream from From c3464dab4d0c7b275945a5fff396d2d88ad935f8 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sat, 8 Oct 2022 10:46:14 -0500 Subject: [PATCH 05/40] starting to clean up Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 58 ++------------------------------ pkg/controller/nodes/executor.go | 15 +++++---- 2 files changed, 11 insertions(+), 62 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index ddafc6b66..1c3427b4f 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -22,21 +22,6 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/status" - - /*"github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" - - "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" - //pluginCore "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/core" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" - "github.com/flyteorg/flytestdlib/logger" - "github.com/pkg/errors" - //"google.golang.org/grpc/codes" - //"google.golang.org/grpc/status" - - "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" - errors2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors"*/ ) func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { @@ -129,44 +114,6 @@ func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx * //t.metrics.reservationGetSuccessCount.Inc(ctx) return catalog.NewReservationEntry(reservation.ExpiresAt.AsTime(), reservation.HeartbeatInterval.AsDuration(), reservation.OwnerId, status), nil - - /*tk, err := tr.Read(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err - } - - if tk.Metadata.Discoverable && tk.Metadata.CacheSerializable { - logger.Infof(ctx, "Catalog CacheSerializeEnabled: creating catalog reservation.") - key := catalog.Key{ - Identifier: *tk.Id, - CacheVersion: tk.Metadata.DiscoveryVersion, - TypedInterface: *tk.Interface, - InputReader: inputReader, - } - - reservation, err := t.catalog.GetOrExtendReservation(ctx, key, ownerID, heartbeatInterval) - if err != nil { - t.metrics.reservationGetFailureCount.Inc(ctx) - logger.Errorf(ctx, "Catalog Failure: reservation get or extend failed. err: %v", err.Error()) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err - } - - expiresAt := reservation.ExpiresAt.AsTime() - heartbeatInterval := reservation.HeartbeatInterval.AsDuration() - - var status core.CatalogReservation_Status - if reservation.OwnerId == ownerID { - status = core.CatalogReservation_RESERVATION_ACQUIRED - } else { - status = core.CatalogReservation_RESERVATION_EXISTS - } - - t.metrics.reservationGetSuccessCount.Inc(ctx) - return catalog.NewReservationEntry(expiresAt, heartbeatInterval, reservation.OwnerId, status), nil - } - logger.Infof(ctx, "Catalog CacheSerializeDisabled: for Task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), nil*/ } // ReleaseCatalogReservation attempts to release an artifact reservation if the task is cachable @@ -212,11 +159,10 @@ func (n *nodeExecutor) WriteCatalogCache(ctx context.Context, nCtx *nodeExecCont return catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), err } - // TODO @hamersaw - is this right? - /*iface := catalogKey.TypedInterface + iface := catalogKey.TypedInterface if iface.Outputs != nil && len(iface.Outputs.Variables) == 0 { return catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), nil - }*/ + } outputPaths := ioutils.NewReadOnlyOutputFilePaths(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetOutputDir()) outputReader := ioutils.NewRemoteFileOutputReader(ctx, nCtx.DataStore(), outputPaths, nCtx.MaxDatasetSizeBytes()) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 8dc8b2e38..6d378e414 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -623,7 +623,8 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut cacheStatus := core.CatalogCacheStatus_CACHE_DISABLED catalogReservationStatus := core.CatalogReservation_RESERVATION_DISABLED if cacheHandler, ok := h.(handler.CacheableNode); ok { - // TODO @hamersaw - we are checking cache twice when nodes start ... need to fix this! + // TODO @hamersaw - document since we already check cache in queued the first time we hit this we shouldn't check it + // could potentially use nodeStatus.GetMessage() check and update on RESERVATION_EXISTS if currentPhase == v1alpha1.NodePhaseQueued { cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { @@ -690,12 +691,14 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut pluginTrns.pInfo = pluginCore.PhaseInfoWaitingForCache(pluginCore.DefaultPhaseVersion, nil) }*/ - if currentPhase == v1alpha1.NodePhaseQueued { + /*if currentPhase == v1alpha1.NodePhaseQueued { //if ts.PluginPhase == pluginCore.PhaseWaitingForCache { logger.Debugf(ctx, "No state change for Task, previously observed same transition. Short circuiting.") - //return executors.NodeStatusQueued, nil // TODO @hamersaw - should probably fallthrough on this - p = handler.PhaseInfoQueued("TODO hamersaw") - } + p = handler.PhaseInfoQueued("waiting on serialized cache") + nodeStatus.UpdatePhase(v1alpha1.NodePhaseQueued, v1.Now(), "waiting on serialized cache", nil) + }*/ + + p = handler.PhaseInfoQueued("node queued") } } } @@ -833,12 +836,12 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) - if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { // process downstream nodes return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } + logger.Infof(ctx, "HAMERSAW end - %v %v", currentPhase, nodeStatus.GetMessage()) return finalStatus, nil } From 3f0b7052001280876438197913a508f4beae3a0d Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 15:35:24 -0500 Subject: [PATCH 06/40] removed commented out code Signed-off-by: Daniel Rammer --- pkg/controller/nodes/dynamic/handler.go | 13 - pkg/controller/nodes/task/handler.go | 108 -------- .../nodes/task/pre_post_execution.go | 257 +----------------- 3 files changed, 5 insertions(+), 373 deletions(-) diff --git a/pkg/controller/nodes/dynamic/handler.go b/pkg/controller/nodes/dynamic/handler.go index 05b599483..4b9a20f14 100644 --- a/pkg/controller/nodes/dynamic/handler.go +++ b/pkg/controller/nodes/dynamic/handler.go @@ -34,9 +34,6 @@ const dynamicNodeID = "dynamic-node" type TaskNodeHandler interface { handler.CacheableNode - /*ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, - r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, - tr ioutils.SimpleTaskReader, m catalog.Metadata) (catalog.Status, *io.ExecutionError, error)*/ ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader) (*io.ExecutionError, error) @@ -149,13 +146,7 @@ func (d dynamicNodeTaskNodeHandler) handleDynamicSubNodes(ctx context.Context, n // These outputPaths only reads the output metadata. So the sandbox is completely optional here and hence it is nil. // The sandbox creation as it uses hashing can be expensive and we skip that expense. outputPaths := ioutils.NewReadOnlyOutputFilePaths(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetOutputDir()) - //execID := task.GetTaskExecutionIdentifier(nCtx) outputReader := ioutils.NewRemoteFileOutputReader(ctx, nCtx.DataStore(), outputPaths, nCtx.MaxDatasetSizeBytes()) - /*status, ee, err := d.TaskNodeHandler.ValidateOutputAndCacheAdd(ctx, nCtx.NodeID(), nCtx.InputReader(), - outputReader, nil, nCtx.ExecutionContext().GetExecutionConfig(), nCtx.TaskReader(), catalog.Metadata{ - TaskExecutionIdentifier: execID, - })*/ - ee, err := d.TaskNodeHandler.ValidateOutput(ctx, nCtx.NodeID(), nCtx.InputReader(), outputReader, nil, nCtx.ExecutionContext().GetExecutionConfig(), nCtx.TaskReader()) @@ -170,10 +161,6 @@ func (d dynamicNodeTaskNodeHandler) handleDynamicSubNodes(ctx context.Context, n return trns.WithInfo(handler.PhaseInfoFailureErr(ee.ExecutionError, trns.Info().GetInfo())), handler.DynamicNodeState{Phase: v1alpha1.DynamicNodePhaseFailing, Reason: ee.ExecutionError.String()}, nil } - // TODO @hamersaw - do we need to even populate this? - //taskNodeInfoMetadata := &event.TaskNodeMetadata{CacheStatus: status.GetCacheStatus(), CatalogKey: status.GetMetadata()} - //trns.WithInfo(trns.Info().WithInfo(&handler.ExecutionInfo{TaskNodeInfo: &handler.TaskNodeInfo{TaskNodeMetadata: taskNodeInfoMetadata}})) - trns.WithInfo(trns.Info().WithInfo(&handler.ExecutionInfo{TaskNodeInfo: &handler.TaskNodeInfo{TaskNodeMetadata: &event.TaskNodeMetadata{}}})) } return trns, newState, nil diff --git a/pkg/controller/nodes/task/handler.go b/pkg/controller/nodes/task/handler.go index eb24a6f65..31e41a703 100644 --- a/pkg/controller/nodes/task/handler.go +++ b/pkg/controller/nodes/task/handler.go @@ -46,17 +46,8 @@ const pluginContextKey = contextutils.Key("plugin") type metrics struct { pluginPanics labeled.Counter unsupportedTaskType labeled.Counter - /*catalogPutFailureCount labeled.Counter - catalogGetFailureCount labeled.Counter - catalogPutSuccessCount labeled.Counter - catalogMissCount labeled.Counter - catalogHitCount labeled.Counter*/ pluginExecutionLatency labeled.StopWatch pluginQueueLatency labeled.StopWatch - /*reservationGetSuccessCount labeled.Counter - reservationGetFailureCount labeled.Counter - reservationReleaseSuccessCount labeled.Counter - reservationReleaseFailureCount labeled.Counter*/ // TODO We should have a metric to capture custom state size scope promutils.Scope @@ -487,11 +478,6 @@ func (t Handler) invokePlugin(ctx context.Context, p pluginCore.Plugin, tCtx *ta // ------------------------------------- logger.Debugf(ctx, "Task success detected, calling on Task success") outputCommitter := ioutils.NewRemoteFileOutputWriter(ctx, tCtx.DataStore(), tCtx.OutputWriter()) - /*execID := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetID() - cacheStatus, ee, err := t.ValidateOutputAndCacheAdd(ctx, tCtx.NodeID(), tCtx.InputReader(), tCtx.ow.GetReader(), - outputCommitter, tCtx.ExecutionContext().GetExecutionConfig(), tCtx.tr, catalog.Metadata{ - TaskExecutionIdentifier: &execID, - })*/ ee, err := t.ValidateOutput(ctx, tCtx.NodeID(), tCtx.InputReader(), tCtx.ow.GetReader(), outputCommitter, tCtx.ExecutionContext().GetExecutionConfig(), tCtx.tr) if err != nil { @@ -540,12 +526,6 @@ func (t Handler) Handle(ctx context.Context, nCtx handler.NodeExecutionContext) return handler.UnknownTransition, errors.Wrapf(errors.UnsupportedTaskTypeError, nCtx.NodeID(), err, "unable to resolve plugin") } - // TODO @hamersaw remove - /*checkCatalog := !p.GetProperties().DisableNodeLevelCaching - if !checkCatalog { - logger.Infof(ctx, "Node level caching is disabled. Skipping catalog read.") - }*/ - tCtx, err := t.newTaskExecutionContext(ctx, nCtx, p) if err != nil { return handler.UnknownTransition, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "unable to create Handler execution context") @@ -561,78 +541,6 @@ func (t Handler) Handle(ctx context.Context, nCtx handler.NodeExecutionContext) // So now we will derive this from the plugin phase // TODO @kumare re-evaluate this decision - // TODO @hamersaw remove - /*// STEP 1: Check Cache - if (ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache) && checkCatalog { - // This is assumed to be first time. we will check catalog and call handle - entry, err := t.CheckCatalogCache(ctx, tCtx.tr, nCtx.InputReader(), tCtx.ow) - if err != nil { - logger.Errorf(ctx, "failed to check catalog cache with error") - return handler.UnknownTransition, err - } - - if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - r := tCtx.ow.GetReader() - if r == nil { - return handler.UnknownTransition, errors.Errorf(errors.IllegalStateError, nCtx.NodeID(), "failed to reader outputs from a CacheHIT. Unexpected!") - } - - // TODO @kumare this can be optimized, if we have paths then the reader could be pipelined to a sink - o, ee, err := r.Read(ctx) - if err != nil { - logger.Errorf(ctx, "failed to read from catalog, err: %s", err.Error()) - return handler.UnknownTransition, err - } - - if ee != nil { - logger.Errorf(ctx, "got execution error from catalog output reader? This should not happen, err: %s", ee.String()) - return handler.UnknownTransition, errors.Errorf(errors.IllegalStateError, nCtx.NodeID(), "execution error from a cache output, bad state: %s", ee.String()) - } - - if err := nCtx.DataStore().WriteProtobuf(ctx, tCtx.ow.GetOutputPath(), storage.Options{}, o); err != nil { - logger.Errorf(ctx, "failed to write cached value to datastore, err: %s", err.Error()) - return handler.UnknownTransition, err - } - - pluginTrns.CacheHit(tCtx.ow.GetOutputPath(), nil, entry) - } else { - logger.Infof(ctx, "No CacheHIT. Status [%s]", entry.GetStatus().GetCacheStatus().String()) - pluginTrns.PopulateCacheInfo(entry) - } - } - - // Check catalog for cache reservation and acquire if none exists - if checkCatalog && (pluginTrns.execInfo.TaskNodeInfo == nil || pluginTrns.execInfo.TaskNodeInfo.TaskNodeMetadata.CacheStatus != core.CatalogCacheStatus_CACHE_HIT) { - ownerID := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetGeneratedName() - reservation, err := t.GetOrExtendCatalogReservation(ctx, ownerID, controllerConfig.GetConfig().WorkflowReEval.Duration, tCtx.tr, nCtx.InputReader()) - if err != nil { - logger.Errorf(ctx, "failed to get or extend catalog reservation with error") - return handler.UnknownTransition, err - } - - pluginTrns.PopulateReservationInfo(reservation) - - if reservation.GetStatus() == core.CatalogReservation_RESERVATION_ACQUIRED && - (ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache) { - logger.Infof(ctx, "Acquired cache reservation") - } - - // If we do not own the reservation then we transition to WaitingForCache phase. If we are - // already running (ie. in a phase other than PhaseUndefined or PhaseWaitingForCache) and - // somehow lost the reservation (ex. by expiration), continue to execute until completion. - if reservation.GetStatus() == core.CatalogReservation_RESERVATION_EXISTS { - if ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache { - pluginTrns.ttype = handler.TransitionTypeEphemeral - pluginTrns.pInfo = pluginCore.PhaseInfoWaitingForCache(pluginCore.DefaultPhaseVersion, nil) - } - - if ts.PluginPhase == pluginCore.PhaseWaitingForCache { - logger.Debugf(ctx, "No state change for Task, previously observed same transition. Short circuiting.") - return pluginTrns.FinalTransition(ctx) - } - } - }*/ - barrierTick := uint32(0) // STEP 2: If no cache-hit and not transitioning to PhaseWaitingForCache, then lets invoke the plugin and wait for a transition out of undefined if pluginTrns.execInfo.TaskNodeInfo == nil || (pluginTrns.pInfo.Phase() != pluginCore.PhaseWaitingForCache && @@ -847,13 +755,6 @@ func (t Handler) Finalize(ctx context.Context, nCtx handler.NodeExecutionContext } }() - // release catalog reservation (if exists) - /*ownerID := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetGeneratedName() - _, err = t.ReleaseCatalogReservation(ctx, ownerID, tCtx.tr, tCtx.InputReader()) - if err != nil { - return errors.Wrapf(errors.CatalogCallFailed, nCtx.NodeID(), err, "failed to release reservation") - }*/ - childCtx := context.WithValue(ctx, pluginContextKey, p.GetID()) err = p.Finalize(childCtx, tCtx) return @@ -880,17 +781,8 @@ func New(ctx context.Context, kubeClient executors.Client, client catalog.Client metrics: &metrics{ pluginPanics: labeled.NewCounter("plugin_panic", "Task plugin paniced when trying to execute a Handler.", scope), unsupportedTaskType: labeled.NewCounter("unsupported_tasktype", "No Handler plugin configured for Handler type", scope), - /*catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", scope), - catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", scope), - catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", scope), - catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", scope), - catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", scope),*/ pluginExecutionLatency: labeled.NewStopWatch("plugin_exec_latency", "Time taken to invoke plugin for one round", time.Microsecond, scope), pluginQueueLatency: labeled.NewStopWatch("plugin_queue_latency", "Time spent by plugin in queued phase", time.Microsecond, scope), - /*reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", scope), - reservationGetSuccessCount: labeled.NewCounter("reservation_get_success_count", "Reservation GetOrExtend success count", scope), - reservationReleaseFailureCount: labeled.NewCounter("reservation_release_failure_count", "Reservation Release failure count", scope), - reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", scope),*/ scope: scope, }, pluginScope: scope.NewSubScope("plugin"), diff --git a/pkg/controller/nodes/task/pre_post_execution.go b/pkg/controller/nodes/task/pre_post_execution.go index 31af864a2..5cf81bed7 100644 --- a/pkg/controller/nodes/task/pre_post_execution.go +++ b/pkg/controller/nodes/task/pre_post_execution.go @@ -2,26 +2,20 @@ package task import ( "context" - //"time" - - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" - //pluginCore "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" - "github.com/flyteorg/flytestdlib/contextutils" - "github.com/flyteorg/flytestdlib/logger" - //"github.com/pkg/errors" - //"google.golang.org/grpc/codes" - //"google.golang.org/grpc/status" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" errors2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" -) -var cacheDisabled = catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil) + "github.com/flyteorg/flytestdlib/contextutils" + "github.com/flyteorg/flytestdlib/logger" +) func (t *Handler) GetCatalogKey(ctx context.Context, nCtx handler.NodeExecutionContext) (catalog.Key, error) { // read task template @@ -105,214 +99,6 @@ func (t *Handler) IsCacheSerializable(ctx context.Context, nCtx handler.NodeExec return taskTemplate.Metadata.Discoverable && taskTemplate.Metadata.CacheSerializable, nil } -/*func (t *Handler) CheckCatalogCache(ctx context.Context, tr pluginCore.TaskReader, inputReader io.InputReader, outputWriter io.OutputWriter) (catalog.Entry, error) { - tk, err := tr.Read(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) - return catalog.Entry{}, err - } - - if tk.Metadata.Discoverable { - logger.Infof(ctx, "Catalog CacheEnabled: Looking up catalog Cache.") - key := catalog.Key{ - Identifier: *tk.Id, - CacheVersion: tk.Metadata.DiscoveryVersion, - TypedInterface: *tk.Interface, - InputReader: inputReader, - } - - resp, err := t.catalog.Get(ctx, key) - if err != nil { - causeErr := errors.Cause(err) - if taskStatus, ok := status.FromError(causeErr); ok && taskStatus.Code() == codes.NotFound { - t.metrics.catalogMissCount.Inc(ctx) - logger.Infof(ctx, "Catalog CacheMiss: Artifact not found in Catalog. Executing Task.") - return catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_MISS, nil)), nil - } - - t.metrics.catalogGetFailureCount.Inc(ctx) - logger.Errorf(ctx, "Catalog Failure: memoization check failed. err: %v", err.Error()) - return catalog.Entry{}, errors.Wrapf(err, "Failed to check Catalog for previous results") - } - - if resp.GetStatus().GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT { - logger.Errorf(ctx, "No CacheHIT and no Error received. Illegal state, Cache State: %s", resp.GetStatus().GetCacheStatus().String()) - // TODO should this be an error? - return resp, nil - } - - logger.Infof(ctx, "Catalog CacheHit: for task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) - t.metrics.catalogHitCount.Inc(ctx) - if iface := tk.Interface; iface != nil && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 { - if err := outputWriter.Put(ctx, resp.GetOutputs()); err != nil { - logger.Errorf(ctx, "failed to write data to Storage, err: %v", err.Error()) - return catalog.Entry{}, errors.Wrapf(err, "failed to copy cached results for task.") - } - } - // SetCached. - return resp, nil - } - logger.Infof(ctx, "Catalog CacheDisabled: for Task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) - return catalog.NewCatalogEntry(nil, cacheDisabled), nil -} - -// GetOrExtendCatalogReservation attempts to acquire an artifact reservation if the task is -// cachable and cache serializable. If the reservation already exists for this owner, the -// reservation is extended. -func (t *Handler) GetOrExtendCatalogReservation(ctx context.Context, ownerID string, heartbeatInterval time.Duration, tr pluginCore.TaskReader, inputReader io.InputReader) (catalog.ReservationEntry, error) { - tk, err := tr.Read(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err - } - - if tk.Metadata.Discoverable && tk.Metadata.CacheSerializable { - logger.Infof(ctx, "Catalog CacheSerializeEnabled: creating catalog reservation.") - key := catalog.Key{ - Identifier: *tk.Id, - CacheVersion: tk.Metadata.DiscoveryVersion, - TypedInterface: *tk.Interface, - InputReader: inputReader, - } - - reservation, err := t.catalog.GetOrExtendReservation(ctx, key, ownerID, heartbeatInterval) - if err != nil { - t.metrics.reservationGetFailureCount.Inc(ctx) - logger.Errorf(ctx, "Catalog Failure: reservation get or extend failed. err: %v", err.Error()) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err - } - - expiresAt := reservation.ExpiresAt.AsTime() - heartbeatInterval := reservation.HeartbeatInterval.AsDuration() - - var status core.CatalogReservation_Status - if reservation.OwnerId == ownerID { - status = core.CatalogReservation_RESERVATION_ACQUIRED - } else { - status = core.CatalogReservation_RESERVATION_EXISTS - } - - t.metrics.reservationGetSuccessCount.Inc(ctx) - return catalog.NewReservationEntry(expiresAt, heartbeatInterval, reservation.OwnerId, status), nil - } - logger.Infof(ctx, "Catalog CacheSerializeDisabled: for Task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), nil -}*/ - -/*func (t *Handler) ValidateOutputAndCacheAdd(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, - r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, - tr ioutils.SimpleTaskReader, m catalog.Metadata) (catalog.Status, *io.ExecutionError, error) { - - tk, err := tr.Read(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) - return cacheDisabled, nil, err - } - - iface := tk.Interface - outputsDeclared := iface != nil && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 - - if r == nil { - if outputsDeclared { - // Whack! plugin did not return any outputs for this task - // Also When an error is observed, cache is automatically disabled - return cacheDisabled, &io.ExecutionError{ - ExecutionError: &core.ExecutionError{ - Code: "OutputsNotGenerated", - Message: "Output Reader was nil. Plugin/Platform problem.", - }, - IsRecoverable: true, - }, nil - } - return cacheDisabled, nil, nil - } - // Reader exists, we can check for error, even if this task may not have any outputs declared - y, err := r.IsError(ctx) - if err != nil { - return cacheDisabled, nil, err - } - if y { - taskErr, err := r.ReadError(ctx) - if err != nil { - return cacheDisabled, nil, err - } - - if taskErr.ExecutionError == nil { - taskErr.ExecutionError = &core.ExecutionError{Kind: core.ExecutionError_UNKNOWN, Code: "Unknown", Message: "Unknown"} - } - return cacheDisabled, &taskErr, nil - } - - // Do this if we have outputs declared for the Handler interface! - if !outputsDeclared { - return cacheDisabled, nil, nil - } - ok, err := r.Exists(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to check if the output file exists. Error: %s", err.Error()) - return cacheDisabled, nil, err - } - - if !ok { - // Does not exist - return cacheDisabled, - &io.ExecutionError{ - ExecutionError: &core.ExecutionError{ - Code: "OutputsNotFound", - Message: "Outputs not generated by task execution", - }, - IsRecoverable: true, - }, nil - } - - if !r.IsFile(ctx) { - // Read output and write to file - // No need to check for Execution Error here as we have done so above this block. - err = outputCommitter.Put(ctx, r) - if err != nil { - logger.Errorf(ctx, "Failed to commit output to remote location. Error: %v", err) - return cacheDisabled, nil, err - } - } - - p, err := t.ResolvePlugin(ctx, tk.Type, executionConfig) - if err != nil { - return cacheDisabled, nil, errors2.Wrapf(errors2.UnsupportedTaskTypeError, nodeID, err, "unable to resolve plugin") - } - writeToCatalog := !p.GetProperties().DisableNodeLevelCaching - - if !tk.Metadata.Discoverable || !writeToCatalog { - if !writeToCatalog { - logger.Infof(ctx, "Node level caching is disabled. Skipping catalog write.") - } - return cacheDisabled, nil, nil - } - - cacheVersion := "0" - if tk.Metadata != nil { - cacheVersion = tk.Metadata.DiscoveryVersion - } - - key := catalog.Key{ - Identifier: *tk.Id, - CacheVersion: cacheVersion, - TypedInterface: *tk.Interface, - InputReader: i, - } - - logger.Infof(ctx, "Catalog CacheEnabled. recording execution [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) - // ignores discovery write failures - s, err2 := t.catalog.Put(ctx, key, r, m) - if err2 != nil { - //t.metrics.catalogPutFailureCount.Inc(ctx) - logger.Errorf(ctx, "Failed to write results to catalog for Task [%v]. Error: %v", tk.GetId(), err2) - return catalog.NewStatus(core.CatalogCacheStatus_CACHE_PUT_FAILURE, s.GetMetadata()), nil, nil - } - //t.metrics.catalogPutSuccessCount.Inc(ctx) - logger.Infof(ctx, "Successfully cached results to catalog - Task [%v]", tk.GetId()) - return s, nil, nil -}*/ - func (t *Handler) ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader) (*io.ExecutionError, error) { @@ -390,36 +176,3 @@ func (t *Handler) ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i return nil, nil } - -/*// ReleaseCatalogReservation attempts to release an artifact reservation if the task is cachable -// and cache serializable. If the reservation does not exist for this owner (e.x. it never existed -// or has been acquired by another owner) this call is still successful. -func (t *Handler) ReleaseCatalogReservation(ctx context.Context, ownerID string, tr pluginCore.TaskReader, inputReader io.InputReader) (catalog.ReservationEntry, error) { - tk, err := tr.Read(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err - } - - if tk.Metadata.Discoverable && tk.Metadata.CacheSerializable { - logger.Infof(ctx, "Catalog CacheSerializeEnabled: releasing catalog reservation.") - key := catalog.Key{ - Identifier: *tk.Id, - CacheVersion: tk.Metadata.DiscoveryVersion, - TypedInterface: *tk.Interface, - InputReader: inputReader, - } - - err := t.catalog.ReleaseReservation(ctx, key, ownerID) - if err != nil { - t.metrics.reservationReleaseFailureCount.Inc(ctx) - logger.Errorf(ctx, "Catalog Failure: release reservation failed. err: %v", err.Error()) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err - } - - t.metrics.reservationReleaseSuccessCount.Inc(ctx) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_RELEASED), nil - } - logger.Infof(ctx, "Catalog CacheSerializeDisabled: for Task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), nil -}*/ From fcb917497273c54e33230bfb16a5de3c1a284cba Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 15:45:17 -0500 Subject: [PATCH 07/40] removed separate IsCacheable and IsCacheSerializable functions from CacheableNode interface Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 132 ++++++++---------- pkg/controller/nodes/handler/iface.go | 3 +- .../nodes/task/pre_post_execution.go | 41 +----- 3 files changed, 69 insertions(+), 107 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 6d378e414..0f6b9b6f9 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -450,11 +450,13 @@ func (c *nodeExecutor) execute(ctx context.Context, h handler.Node, nCtx *nodeEx // TODO @hamersaw - complete if phase.GetPhase() == handler.EPhaseSuccess { if cacheHandler, ok := h.(handler.CacheableNode); ok { - cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) + cacheable, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) return handler.PhaseInfoUndefined, err - } else if cacheable { + } + + if cacheable { status, err := c.WriteCatalogCache(ctx, nCtx, cacheHandler) if err != nil { // TODO @hamersaw fail @@ -481,11 +483,7 @@ func (c *nodeExecutor) execute(ctx context.Context, h handler.Node, nCtx *nodeEx phase = phase.WithInfo(info) } - cacheSerializable, err := cacheHandler.IsCacheSerializable(ctx, nCtx) - if err != nil { - logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) - return handler.PhaseInfoUndefined, err - } else if cacheSerializable { + if cacheSerializable { _, err := c.ReleaseCatalogReservation(ctx, nCtx, cacheHandler) if err != nil { // TODO @hamersaw fail @@ -537,7 +535,7 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor // TODO @hamersaw - complete cacheStatus := core.CatalogCacheStatus_CACHE_DISABLED if cacheHandler, ok := h.(handler.CacheableNode); ok { - cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) + cacheable, _, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) return executors.NodeStatusUndefined, err @@ -623,83 +621,77 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut cacheStatus := core.CatalogCacheStatus_CACHE_DISABLED catalogReservationStatus := core.CatalogReservation_RESERVATION_DISABLED if cacheHandler, ok := h.(handler.CacheableNode); ok { + cacheable, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return executors.NodeStatusUndefined, err + } + // TODO @hamersaw - document since we already check cache in queued the first time we hit this we shouldn't check it // could potentially use nodeStatus.GetMessage() check and update on RESERVATION_EXISTS - if currentPhase == v1alpha1.NodePhaseQueued { - cacheable, err := cacheHandler.IsCacheable(ctx, nCtx) + if cacheable && currentPhase == v1alpha1.NodePhaseQueued { + entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) if err != nil { - logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + // TODO @hamersaw fail return executors.NodeStatusUndefined, err - } else if cacheable { - entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) - if err != nil { - // TODO @hamersaw fail - return executors.NodeStatusUndefined, err - } + } - cacheStatus = entry.GetStatus().GetCacheStatus() - if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { - // update NodeStatus to Success - nodeStatus.ClearSubNodeStatus() - nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) - - // set phaseInfo transition to include ... TODO @hamersaw - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) - p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ - OutputInfo: &handler.OutputInfo { - OutputURI: outputFile, - }, - TaskNodeInfo: &handler.TaskNodeInfo { - TaskNodeMetadata: &event.TaskNodeMetadata{ - CacheStatus: entry.GetStatus().GetCacheStatus(), - CatalogKey: entry.GetStatus().GetMetadata(), - }, + cacheStatus = entry.GetStatus().GetCacheStatus() + if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { + // update NodeStatus to Success + nodeStatus.ClearSubNodeStatus() + nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) + + // set phaseInfo transition to include ... TODO @hamersaw + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ + OutputInfo: &handler.OutputInfo { + OutputURI: outputFile, + }, + TaskNodeInfo: &handler.TaskNodeInfo { + TaskNodeMetadata: &event.TaskNodeMetadata{ + CacheStatus: entry.GetStatus().GetCacheStatus(), + CatalogKey: entry.GetStatus().GetMetadata(), }, - }) - } + }, + }) } } - if cacheStatus != core.CatalogCacheStatus_CACHE_HIT { - cacheSerializable, err := cacheHandler.IsCacheSerializable(ctx, nCtx) + if cacheSerializable && cacheStatus != core.CatalogCacheStatus_CACHE_HIT { + entry, err := c.GetOrExtendCatalogReservation(ctx, nCtx, cacheHandler, config.GetConfig().WorkflowReEval.Duration) if err != nil { - logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + // TODO @hamersaw fail return executors.NodeStatusUndefined, err - } else if cacheSerializable { - entry, err := c.GetOrExtendCatalogReservation(ctx, nCtx, cacheHandler, config.GetConfig().WorkflowReEval.Duration) - if err != nil { - // TODO @hamersaw fail - return executors.NodeStatusUndefined, err - } + } - // TODO @hamersaw - update phase info with reservation info - //p.execInfo.TaskNodeInfo.TaskNodeMetadata.ReservationStatus = entry.GetStatus() - //pluginTrns.PopulateReservationInfo(reservation) + // TODO @hamersaw - update phase info with reservation info + //p.execInfo.TaskNodeInfo.TaskNodeMetadata.ReservationStatus = entry.GetStatus() + //pluginTrns.PopulateReservationInfo(reservation) + + /*if reservation.GetStatus() == core.CatalogReservation_RESERVATION_ACQUIRED && + (ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache) { + logger.Infof(ctx, "Acquired cache reservation") + }*/ + + // If we do not own the reservation then we transition to WaitingForCache phase. If we are + // already running (ie. in a phase other than PhaseUndefined or PhaseWaitingForCache) and + // somehow lost the reservation (ex. by expiration), continue to execute until completion. + catalogReservationStatus = entry.GetStatus() + if catalogReservationStatus == core.CatalogReservation_RESERVATION_EXISTS { + /*if ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache { + pluginTrns.ttype = handler.TransitionTypeEphemeral + pluginTrns.pInfo = pluginCore.PhaseInfoWaitingForCache(pluginCore.DefaultPhaseVersion, nil) + }*/ - /*if reservation.GetStatus() == core.CatalogReservation_RESERVATION_ACQUIRED && - (ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache) { - logger.Infof(ctx, "Acquired cache reservation") + /*if currentPhase == v1alpha1.NodePhaseQueued { + //if ts.PluginPhase == pluginCore.PhaseWaitingForCache { + logger.Debugf(ctx, "No state change for Task, previously observed same transition. Short circuiting.") + p = handler.PhaseInfoQueued("waiting on serialized cache") + nodeStatus.UpdatePhase(v1alpha1.NodePhaseQueued, v1.Now(), "waiting on serialized cache", nil) }*/ - // If we do not own the reservation then we transition to WaitingForCache phase. If we are - // already running (ie. in a phase other than PhaseUndefined or PhaseWaitingForCache) and - // somehow lost the reservation (ex. by expiration), continue to execute until completion. - catalogReservationStatus = entry.GetStatus() - if catalogReservationStatus == core.CatalogReservation_RESERVATION_EXISTS { - /*if ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache { - pluginTrns.ttype = handler.TransitionTypeEphemeral - pluginTrns.pInfo = pluginCore.PhaseInfoWaitingForCache(pluginCore.DefaultPhaseVersion, nil) - }*/ - - /*if currentPhase == v1alpha1.NodePhaseQueued { - //if ts.PluginPhase == pluginCore.PhaseWaitingForCache { - logger.Debugf(ctx, "No state change for Task, previously observed same transition. Short circuiting.") - p = handler.PhaseInfoQueued("waiting on serialized cache") - nodeStatus.UpdatePhase(v1alpha1.NodePhaseQueued, v1.Now(), "waiting on serialized cache", nil) - }*/ - - p = handler.PhaseInfoQueued("node queued") - } + p = handler.PhaseInfoQueued("node queued") } } } diff --git a/pkg/controller/nodes/handler/iface.go b/pkg/controller/nodes/handler/iface.go index 8dd752b0e..d1c05e910 100644 --- a/pkg/controller/nodes/handler/iface.go +++ b/pkg/controller/nodes/handler/iface.go @@ -31,6 +31,5 @@ type Node interface { type CacheableNode interface { Node GetCatalogKey(ctx context.Context, executionContext NodeExecutionContext) (catalog.Key, error) - IsCacheable(ctx context.Context, executionContext NodeExecutionContext) (bool, error) - IsCacheSerializable(ctx context.Context, executionContext NodeExecutionContext) (bool, error) + IsCacheable(ctx context.Context, executionContext NodeExecutionContext) (bool, bool, error) } diff --git a/pkg/controller/nodes/task/pre_post_execution.go b/pkg/controller/nodes/task/pre_post_execution.go index 5cf81bed7..e7439c2c2 100644 --- a/pkg/controller/nodes/task/pre_post_execution.go +++ b/pkg/controller/nodes/task/pre_post_execution.go @@ -39,64 +39,35 @@ func (t *Handler) GetCatalogKey(ctx context.Context, nCtx handler.NodeExecutionC }, nil } -func (t *Handler) IsCacheable(ctx context.Context, nCtx handler.NodeExecutionContext) (bool, error) { +func (t *Handler) IsCacheable(ctx context.Context, nCtx handler.NodeExecutionContext) (bool, bool, error) { // check if plugin has caching disabled ttype := nCtx.TaskReader().GetTaskType() ctx = contextutils.WithTaskType(ctx, ttype) p, err := t.ResolvePlugin(ctx, ttype, nCtx.ExecutionContext().GetExecutionConfig()) if err != nil { - return false, errors2.Wrapf(errors2.UnsupportedTaskTypeError, nCtx.NodeID(), err, "unable to resolve plugin") + return false, false, errors2.Wrapf(errors2.UnsupportedTaskTypeError, nCtx.NodeID(), err, "unable to resolve plugin") } checkCatalog := !p.GetProperties().DisableNodeLevelCaching if !checkCatalog { logger.Infof(ctx, "Node level caching is disabled. Skipping catalog read.") + return false, false, nil } // read task template taskTemplatePath, err := ioutils.GetTaskTemplatePath(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetDataDir()) if err != nil { - return false, err + return false, false, err } taskReader := ioutils.NewLazyUploadingTaskReader(nCtx.TaskReader(), taskTemplatePath, nCtx.DataStore()) taskTemplate, err := taskReader.Read(ctx) if err != nil { logger.Errorf(ctx, "failed to read TaskTemplate, error :%s", err.Error()) - return false, err + return false, false, err } - return taskTemplate.Metadata.Discoverable, nil -} - -func (t *Handler) IsCacheSerializable(ctx context.Context, nCtx handler.NodeExecutionContext) (bool, error) { - // check if plugin has caching disabled - ttype := nCtx.TaskReader().GetTaskType() - ctx = contextutils.WithTaskType(ctx, ttype) - p, err := t.ResolvePlugin(ctx, ttype, nCtx.ExecutionContext().GetExecutionConfig()) - if err != nil { - return false, errors2.Wrapf(errors2.UnsupportedTaskTypeError, nCtx.NodeID(), err, "unable to resolve plugin") - } - - checkCatalog := !p.GetProperties().DisableNodeLevelCaching - if !checkCatalog { - logger.Infof(ctx, "Node level caching is disabled. Skipping catalog read.") - } - - // read task template - taskTemplatePath, err := ioutils.GetTaskTemplatePath(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetDataDir()) - if err != nil { - return false, err - } - - taskReader := ioutils.NewLazyUploadingTaskReader(nCtx.TaskReader(), taskTemplatePath, nCtx.DataStore()) - taskTemplate, err := taskReader.Read(ctx) - if err != nil { - logger.Errorf(ctx, "failed to read TaskTemplate, error :%s", err.Error()) - return false, err - } - - return taskTemplate.Metadata.Discoverable && taskTemplate.Metadata.CacheSerializable, nil + return taskTemplate.Metadata.Discoverable, taskTemplate.Metadata.Discoverable && taskTemplate.Metadata.CacheSerializable, nil } func (t *Handler) ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, From 46b525ebc005e41100cb5addc3cd3c0e79797108 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 16:02:27 -0500 Subject: [PATCH 08/40] refactored reservation owner id to new function to remove duplication Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 32 ++++++++++++++++---------------- pkg/controller/nodes/executor.go | 12 ++++++++---- 2 files changed, 24 insertions(+), 20 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 1c3427b4f..4eef09cf8 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -24,6 +24,20 @@ import ( "google.golang.org/grpc/status" ) +func computeCatalogReservationOwnerID(ctx context.Context, nCtx *nodeExecContext) (string, error) { + currentNodeUniqueID, err := common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) + if err != nil { + return "", err + } + + _, ownerID, err := task.ComputeRawOutputPrefix(ctx, task.IDMaxLength, nCtx, currentNodeUniqueID, nCtx.CurrentAttempt()) + if err != nil { + return "", err + } + + return ownerID, nil +} + func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { @@ -84,14 +98,7 @@ func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx * return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err } - // compute ownerID - currentNodeUniqueID, err := common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) - if err != nil { - // TODO @hamersaw - fail - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err - } - - _, ownerID, err := task.ComputeRawOutputPrefix(ctx, task.IDMaxLength, nCtx, currentNodeUniqueID, nCtx.CurrentAttempt()) + ownerID, err := computeCatalogReservationOwnerID(ctx, nCtx) if err != nil { // TODO @hamersaw - fail return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err @@ -128,14 +135,7 @@ func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *node return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err } - // compute ownerID - TODO @hamersaw make separate function (duplicated code) - currentNodeUniqueID, err := common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) - if err != nil { - // TODO @hamersaw - fail - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err - } - - _, ownerID, err := task.ComputeRawOutputPrefix(ctx, task.IDMaxLength, nCtx, currentNodeUniqueID, nCtx.CurrentAttempt()) + ownerID, err := computeCatalogReservationOwnerID(ctx, nCtx) if err != nil { // TODO @hamersaw - fail return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 0f6b9b6f9..c9e337819 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -638,9 +638,9 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut cacheStatus = entry.GetStatus().GetCacheStatus() if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { - // update NodeStatus to Success - nodeStatus.ClearSubNodeStatus() - nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) + // update NodeStatus to Success // TODO @hamersaw - validate that we don't need to do + //nodeStatus.ClearSubNodeStatus() + //nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) // set phaseInfo transition to include ... TODO @hamersaw outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) @@ -691,7 +691,7 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut nodeStatus.UpdatePhase(v1alpha1.NodePhaseQueued, v1.Now(), "waiting on serialized cache", nil) }*/ - p = handler.PhaseInfoQueued("node queued") + p = handler.PhaseInfoQueued("waiting on serialized cache") } } } @@ -701,6 +701,10 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut lastAttemptStartTime := nodeStatus.GetLastAttemptStartedAt() // TODO @hamersaw - document + // a few scenarios we need to cover + // - cache hit + // - waiting on cache serialize + // - lost the cache reservation, but already running - should not happen. but should still progress if currentPhase != v1alpha1.NodePhaseQueued || (cacheStatus != core.CatalogCacheStatus_CACHE_HIT && catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS) { var err error From 82a76c5aa50db55daa8472983a1c25bc58ae82b7 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 16:51:37 -0500 Subject: [PATCH 09/40] added cache metrics to the node executor Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 32 ++++++++------ pkg/controller/nodes/executor.go | 59 ++++++++++++++++--------- pkg/controller/nodes/handler_factory.go | 2 +- 3 files changed, 59 insertions(+), 34 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 4eef09cf8..2e4b9eddd 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -45,24 +45,31 @@ func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecCont return catalog.Entry{}, err } - //logger.Infof(ctx, "Catalog CacheEnabled: Looking up catalog Cache.") entry, err := n.catalog.Get(ctx, catalogKey) if err != nil { causeErr := errors.Cause(err) if taskStatus, ok := status.FromError(causeErr); ok && taskStatus.Code() == codes.NotFound { - //t.metrics.catalogMissCount.Inc(ctx) + n.metrics.catalogMissCount.Inc(ctx) logger.Infof(ctx, "Catalog CacheMiss: Artifact not found in Catalog. Executing Task.") return catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_MISS, nil)), nil } - //t.metrics.catalogGetFailureCount.Inc(ctx) + n.metrics.catalogGetFailureCount.Inc(ctx) logger.Errorf(ctx, "Catalog Failure: memoization check failed. err: %v", err.Error()) return catalog.Entry{}, errors.Wrapf(err, "Failed to check Catalog for previous results") } - // TODO @hamersaw - figure out + if entry.GetStatus().GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT { + logger.Errorf(ctx, "No CacheHIT and no Error received. Illegal state, Cache State: %s", entry.GetStatus().GetCacheStatus().String()) + // TODO should this be an error? + return entry, nil + } + + //logger.Infof(ctx, "Catalog CacheHit: for task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) + n.metrics.catalogHitCount.Inc(ctx) + iface := catalogKey.TypedInterface - if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 { + if iface.Outputs != nil && len(iface.Outputs.Variables) > 0 { // copy cached outputs to node outputs o, ee, err := entry.GetOutputs().Read(ctx) if err != nil { @@ -80,9 +87,6 @@ func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecCont } } - // SetCached. - //logger.Errorf(ctx, "No CacheHIT and no Error received. Illegal state, Cache State: %s", entry.GetStatus().GetCacheStatus().String()) - // TODO should this be an error? return entry, nil } @@ -106,7 +110,7 @@ func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx * reservation, err := n.catalog.GetOrExtendReservation(ctx, catalogKey, ownerID, heartbeatInterval) if err != nil { - //t.metrics.reservationGetFailureCount.Inc(ctx) + n.metrics.reservationGetFailureCount.Inc(ctx) logger.Errorf(ctx, "Catalog Failure: reservation get or extend failed. err: %v", err.Error()) return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err } @@ -118,7 +122,7 @@ func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx * status = core.CatalogReservation_RESERVATION_EXISTS } - //t.metrics.reservationGetSuccessCount.Inc(ctx) + n.metrics.reservationGetSuccessCount.Inc(ctx) return catalog.NewReservationEntry(reservation.ExpiresAt.AsTime(), reservation.HeartbeatInterval.AsDuration(), reservation.OwnerId, status), nil } @@ -143,12 +147,12 @@ func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *node err = n.catalog.ReleaseReservation(ctx, catalogKey, ownerID) if err != nil { - //t.metrics.reservationReleaseFailureCount.Inc(ctx) + n.metrics.reservationReleaseFailureCount.Inc(ctx) logger.Errorf(ctx, "Catalog Failure: release reservation failed. err: %v", err.Error()) return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_FAILURE), err } - //t.metrics.reservationReleaseSuccessCount.Inc(ctx) + n.metrics.reservationReleaseSuccessCount.Inc(ctx) return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_RELEASED), nil } @@ -176,10 +180,12 @@ func (n *nodeExecutor) WriteCatalogCache(ctx context.Context, nCtx *nodeExecCont // ignores discovery write failures status, err := n.catalog.Put(ctx, catalogKey, outputReader, metadata) if err != nil { - //t.metrics.catalogPutFailureCount.Inc(ctx) + n.metrics.catalogPutFailureCount.Inc(ctx) //logger.Errorf(ctx, "Failed to write results to catalog for Task [%v]. Error: %v", tk.GetId(), err2) return catalog.NewStatus(core.CatalogCacheStatus_CACHE_PUT_FAILURE, status.GetMetadata()), nil } + n.metrics.catalogPutSuccessCount.Inc(ctx) + //logger.Infof(ctx, "Successfully cached results to catalog - Task [%v]", tk.GetId()) return status, nil } diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index c9e337819..26cce6850 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -82,6 +82,16 @@ type nodeMetrics struct { QueuingLatency labeled.StopWatch NodeExecutionTime labeled.StopWatch NodeInputGatherLatency labeled.StopWatch + + catalogPutFailureCount labeled.Counter + catalogGetFailureCount labeled.Counter + catalogPutSuccessCount labeled.Counter + catalogMissCount labeled.Counter + catalogHitCount labeled.Counter + reservationGetSuccessCount labeled.Counter + reservationGetFailureCount labeled.Counter + reservationReleaseSuccessCount labeled.Counter + reservationReleaseFailureCount labeled.Counter } // Implements the executors.Node interface @@ -1356,26 +1366,35 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora taskRecorder: events.NewTaskEventRecorder(eventSink, scope.NewSubScope("task"), store), maxDatasetSizeBytes: maxDatasetSize, metrics: &nodeMetrics{ - Scope: nodeScope, - FailureDuration: labeled.NewStopWatch("failure_duration", "Indicates the total execution time of a failed workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - SuccessDuration: labeled.NewStopWatch("success_duration", "Indicates the total execution time of a successful workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - RecoveryDuration: labeled.NewStopWatch("recovery_duration", "Indicates the total execution time of a recovered workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - UserErrorDuration: labeled.NewStopWatch("user_error_duration", "Indicates the total execution time before user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - SystemErrorDuration: labeled.NewStopWatch("system_error_duration", "Indicates the total execution time before system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - UnknownErrorDuration: labeled.NewStopWatch("unknown_error_duration", "Indicates the total execution time before unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - PermanentUserErrorDuration: labeled.NewStopWatch("perma_user_error_duration", "Indicates the total execution time before non recoverable user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - PermanentSystemErrorDuration: labeled.NewStopWatch("perma_system_error_duration", "Indicates the total execution time before non recoverable system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - PermanentUnknownErrorDuration: labeled.NewStopWatch("perma_unknown_error_duration", "Indicates the total execution time before non recoverable unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - InputsWriteFailure: labeled.NewCounter("inputs_write_fail", "Indicates failure in writing node inputs to metastore", nodeScope), - TimedOutFailure: labeled.NewCounter("timeout_fail", "Indicates failure due to timeout", nodeScope), - InterruptedThresholdHit: labeled.NewCounter("interrupted_threshold", "Indicates the node interruptible disabled because it hit max failure count", nodeScope), - InterruptibleNodesRunning: labeled.NewCounter("interruptible_nodes_running", "number of interruptible nodes running", nodeScope), - InterruptibleNodesTerminated: labeled.NewCounter("interruptible_nodes_terminated", "number of interruptible nodes finished running", nodeScope), - ResolutionFailure: labeled.NewCounter("input_resolve_fail", "Indicates failure in resolving node inputs", nodeScope), - TransitionLatency: labeled.NewStopWatch("transition_latency", "Measures the latency between the last parent node stoppedAt time and current node's queued time.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - QueuingLatency: labeled.NewStopWatch("queueing_latency", "Measures the latency between the time a node's been queued to the time the handler reported the executable moved to running state", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - NodeExecutionTime: labeled.NewStopWatch("node_exec_latency", "Measures the time taken to execute one node, a node can be complex so it may encompass sub-node latency.", time.Microsecond, nodeScope, labeled.EmitUnlabeledMetric), - NodeInputGatherLatency: labeled.NewStopWatch("node_input_latency", "Measures the latency to aggregate inputs and check readiness of a node", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + Scope: nodeScope, + FailureDuration: labeled.NewStopWatch("failure_duration", "Indicates the total execution time of a failed workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + SuccessDuration: labeled.NewStopWatch("success_duration", "Indicates the total execution time of a successful workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + RecoveryDuration: labeled.NewStopWatch("recovery_duration", "Indicates the total execution time of a recovered workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + UserErrorDuration: labeled.NewStopWatch("user_error_duration", "Indicates the total execution time before user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + SystemErrorDuration: labeled.NewStopWatch("system_error_duration", "Indicates the total execution time before system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + UnknownErrorDuration: labeled.NewStopWatch("unknown_error_duration", "Indicates the total execution time before unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + PermanentUserErrorDuration: labeled.NewStopWatch("perma_user_error_duration", "Indicates the total execution time before non recoverable user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + PermanentSystemErrorDuration: labeled.NewStopWatch("perma_system_error_duration", "Indicates the total execution time before non recoverable system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + PermanentUnknownErrorDuration: labeled.NewStopWatch("perma_unknown_error_duration", "Indicates the total execution time before non recoverable unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + InputsWriteFailure: labeled.NewCounter("inputs_write_fail", "Indicates failure in writing node inputs to metastore", nodeScope), + TimedOutFailure: labeled.NewCounter("timeout_fail", "Indicates failure due to timeout", nodeScope), + InterruptedThresholdHit: labeled.NewCounter("interrupted_threshold", "Indicates the node interruptible disabled because it hit max failure count", nodeScope), + InterruptibleNodesRunning: labeled.NewCounter("interruptible_nodes_running", "number of interruptible nodes running", nodeScope), + InterruptibleNodesTerminated: labeled.NewCounter("interruptible_nodes_terminated", "number of interruptible nodes finished running", nodeScope), + ResolutionFailure: labeled.NewCounter("input_resolve_fail", "Indicates failure in resolving node inputs", nodeScope), + TransitionLatency: labeled.NewStopWatch("transition_latency", "Measures the latency between the last parent node stoppedAt time and current node's queued time.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + QueuingLatency: labeled.NewStopWatch("queueing_latency", "Measures the latency between the time a node's been queued to the time the handler reported the executable moved to running state", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + NodeExecutionTime: labeled.NewStopWatch("node_exec_latency", "Measures the time taken to execute one node, a node can be complex so it may encompass sub-node latency.", time.Microsecond, nodeScope, labeled.EmitUnlabeledMetric), + NodeInputGatherLatency: labeled.NewStopWatch("node_input_latency", "Measures the latency to aggregate inputs and check readiness of a node", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", scope), + catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", scope), + catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", scope), + catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", scope), + catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", scope), + reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", scope), + reservationGetSuccessCount: labeled.NewCounter("reservation_get_success_count", "Reservation GetOrExtend success count", scope), + reservationReleaseFailureCount: labeled.NewCounter("reservation_release_failure_count", "Reservation Release failure count", scope), + reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", scope), }, outputResolver: NewRemoteFileOutputResolver(store), defaultExecutionDeadline: nodeConfig.DefaultDeadlines.DefaultNodeExecutionDeadline.Duration, diff --git a/pkg/controller/nodes/handler_factory.go b/pkg/controller/nodes/handler_factory.go index 237e2459c..72e361070 100644 --- a/pkg/controller/nodes/handler_factory.go +++ b/pkg/controller/nodes/handler_factory.go @@ -58,7 +58,7 @@ func NewHandlerFactory(ctx context.Context, executor executors.Node, workflowLau launchPlanReader launchplan.Reader, kubeClient executors.Client, client catalog.Client, recoveryClient recovery.Client, eventConfig *config.EventConfig, clusterID string, scope promutils.Scope) (HandlerFactory, error) { - t, err := task.New(ctx, kubeClient, client ,eventConfig, clusterID, scope) + t, err := task.New(ctx, kubeClient, client, eventConfig, clusterID, scope) if err != nil { return nil, err } From 1a33a4dca7b886c088ec32f1ddf268d7e0b4e9e4 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 17:12:43 -0500 Subject: [PATCH 10/40] cleaned up node cache.go Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 42 ++++++++++++++++------------ pkg/controller/nodes/task/handler.go | 8 +++--- 2 files changed, 28 insertions(+), 22 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 2e4b9eddd..389f4d198 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -24,6 +24,9 @@ import ( "google.golang.org/grpc/status" ) +// computeCatalogReservationOwnerID constructs a unique identifier which includes the nodes +// parent information, node ID, and retry attempt number. This is used to uniquely identify a task +// when the cache reservation API to serialize cached executions. func computeCatalogReservationOwnerID(ctx context.Context, nCtx *nodeExecContext) (string, error) { currentNodeUniqueID, err := common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) if err != nil { @@ -38,11 +41,12 @@ func computeCatalogReservationOwnerID(ctx context.Context, nCtx *nodeExecContext return ownerID, nil } +// CheckCatalogCache uses the handler and contexts to check if cached outputs for the current node +// exist. If the exist, this function also copies the outputs to this node. func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { - // TODO @hamersaw fail - return catalog.Entry{}, err + return catalog.Entry{}, errors.Wrapf(err, "failed to initialize the catalogKey") } entry, err := n.catalog.Get(ctx, catalogKey) @@ -65,7 +69,8 @@ func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecCont return entry, nil } - //logger.Infof(ctx, "Catalog CacheHit: for task [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) + logger.Infof(ctx, "Catalog CacheHit: for task [%s/%s/%s/%s]", catalogKey.Identifier.Project, + catalogKey.Identifier.Domain, catalogKey.Identifier.Name, catalogKey.Identifier.Version) n.metrics.catalogHitCount.Inc(ctx) iface := catalogKey.TypedInterface @@ -98,14 +103,14 @@ func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx * catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { - // TODO @hamersaw fail - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), + errors.Wrapf(err, "failed to initialize the catalogKey") } ownerID, err := computeCatalogReservationOwnerID(ctx, nCtx) if err != nil { - // TODO @hamersaw - fail - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), + errors.Wrapf(err, "failed to initialize the cache reservation ownerID") } reservation, err := n.catalog.GetOrExtendReservation(ctx, catalogKey, ownerID, heartbeatInterval) @@ -135,14 +140,14 @@ func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *node catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { - // TODO @hamersaw fail - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), + errors.Wrapf(err, "failed to initialize the catalogKey") } ownerID, err := computeCatalogReservationOwnerID(ctx, nCtx) if err != nil { - // TODO @hamersaw - fail - return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), err + return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), + errors.Wrapf(err, "failed to initialize the cache reservation ownerID") } err = n.catalog.ReleaseReservation(ctx, catalogKey, ownerID) @@ -156,11 +161,12 @@ func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *node return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_RELEASED), nil } +// WriteCatalogCache relays the outputs of this node to the cache. This allows future executions +// to reuse these data to avoid recomputation. func (n *nodeExecutor) WriteCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Status, error) { catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { - // TODO @hamersaw fail - return catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), err + return catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), errors.Wrapf(err, "failed to initialize the catalogKey") } iface := catalogKey.TypedInterface @@ -168,24 +174,24 @@ func (n *nodeExecutor) WriteCatalogCache(ctx context.Context, nCtx *nodeExecCont return catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), nil } + logger.Infof(ctx, "Catalog CacheEnabled. recording execution [%s/%s/%s/%s]", catalogKey.Identifier.Project, + catalogKey.Identifier.Domain, catalogKey.Identifier.Name, catalogKey.Identifier.Version) + outputPaths := ioutils.NewReadOnlyOutputFilePaths(ctx, nCtx.DataStore(), nCtx.NodeStatus().GetOutputDir()) outputReader := ioutils.NewRemoteFileOutputReader(ctx, nCtx.DataStore(), outputPaths, nCtx.MaxDatasetSizeBytes()) - - // TODO @hamersaw - need to update this once we support caching of non-tasks metadata := catalog.Metadata{ TaskExecutionIdentifier: task.GetTaskExecutionIdentifier(nCtx), } - //logger.Infof(ctx, "Catalog CacheEnabled. recording execution [%s/%s/%s/%s]", tk.Id.Project, tk.Id.Domain, tk.Id.Name, tk.Id.Version) // ignores discovery write failures status, err := n.catalog.Put(ctx, catalogKey, outputReader, metadata) if err != nil { n.metrics.catalogPutFailureCount.Inc(ctx) - //logger.Errorf(ctx, "Failed to write results to catalog for Task [%v]. Error: %v", tk.GetId(), err2) + logger.Errorf(ctx, "Failed to write results to catalog for Task [%v]. Error: %v", catalogKey.Identifier, err) return catalog.NewStatus(core.CatalogCacheStatus_CACHE_PUT_FAILURE, status.GetMetadata()), nil } n.metrics.catalogPutSuccessCount.Inc(ctx) - //logger.Infof(ctx, "Successfully cached results to catalog - Task [%v]", tk.GetId()) + logger.Infof(ctx, "Successfully cached results to catalog - Task [%v]", catalogKey.Identifier) return status, nil } diff --git a/pkg/controller/nodes/task/handler.go b/pkg/controller/nodes/task/handler.go index 31e41a703..99a2952f8 100644 --- a/pkg/controller/nodes/task/handler.go +++ b/pkg/controller/nodes/task/handler.go @@ -44,10 +44,10 @@ import ( const pluginContextKey = contextutils.Key("plugin") type metrics struct { - pluginPanics labeled.Counter - unsupportedTaskType labeled.Counter - pluginExecutionLatency labeled.StopWatch - pluginQueueLatency labeled.StopWatch + pluginPanics labeled.Counter + unsupportedTaskType labeled.Counter + pluginExecutionLatency labeled.StopWatch + pluginQueueLatency labeled.StopWatch // TODO We should have a metric to capture custom state size scope promutils.Scope From 783858a9065f7d2d5f238b273f61aa49bd81c6bd Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 17:15:52 -0500 Subject: [PATCH 11/40] more cleanup Signed-off-by: Daniel Rammer --- pkg/controller/nodes/dynamic/handler.go | 1 - pkg/controller/nodes/handler/iface.go | 7 ++++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/controller/nodes/dynamic/handler.go b/pkg/controller/nodes/dynamic/handler.go index 4b9a20f14..d247470c1 100644 --- a/pkg/controller/nodes/dynamic/handler.go +++ b/pkg/controller/nodes/dynamic/handler.go @@ -7,7 +7,6 @@ import ( "github.com/flyteorg/flytepropeller/pkg/controller/config" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" - //"github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flytestdlib/logger" diff --git a/pkg/controller/nodes/handler/iface.go b/pkg/controller/nodes/handler/iface.go index d1c05e910..165147d8a 100644 --- a/pkg/controller/nodes/handler/iface.go +++ b/pkg/controller/nodes/handler/iface.go @@ -27,9 +27,14 @@ type Node interface { Finalize(ctx context.Context, executionContext NodeExecutionContext) error } -// TODO @hamersaw document +// CacheableNode is a node that supports caching type CacheableNode interface { Node + + // GetCatalogKey returns the unique key for the node represented by the NodeExecutionContext GetCatalogKey(ctx context.Context, executionContext NodeExecutionContext) (catalog.Key, error) + + // IsCacheable returns two booleans representing if the node represented by the + // NodeExecutionContext is cacheable and cache serializable respectively. IsCacheable(ctx context.Context, executionContext NodeExecutionContext) (bool, bool, error) } From 43c68275cbdae85c09588df046f53378a620cbfb Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 20:31:46 -0500 Subject: [PATCH 12/40] setting cache information in phase info so that it is available in events Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 31 ++++++ pkg/controller/nodes/executor.go | 174 ++++++++++++------------------- 2 files changed, 98 insertions(+), 107 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 389f4d198..a778d6b58 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -5,6 +5,7 @@ import ( "time" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" @@ -41,6 +42,36 @@ func computeCatalogReservationOwnerID(ctx context.Context, nCtx *nodeExecContext return ownerID, nil } +func updatePhaseCacheInfo(phaseInfo handler.PhaseInfo, cacheStatus *catalog.Status, reservationStatus *core.CatalogReservation_Status) handler.PhaseInfo { + if cacheStatus == nil && reservationStatus == nil { + return phaseInfo + } + + info := phaseInfo.GetInfo() + if info == nil { + info = &handler.ExecutionInfo{} + } + + if info.TaskNodeInfo == nil { + info.TaskNodeInfo = &handler.TaskNodeInfo{} + } + + if info.TaskNodeInfo.TaskNodeMetadata == nil { + info.TaskNodeInfo.TaskNodeMetadata = &event.TaskNodeMetadata{} + } + + if cacheStatus != nil { + info.TaskNodeInfo.TaskNodeMetadata.CacheStatus = cacheStatus.GetCacheStatus() + info.TaskNodeInfo.TaskNodeMetadata.CatalogKey = cacheStatus.GetMetadata() + } + + if reservationStatus != nil { + info.TaskNodeInfo.TaskNodeMetadata.ReservationStatus = *reservationStatus + } + + return phaseInfo.WithInfo(info) +} + // CheckCatalogCache uses the handler and contexts to check if cached outputs for the current node // exist. If the exist, this function also copies the outputs to this node. func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 26cce6850..d3b764897 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -457,52 +457,6 @@ func (c *nodeExecutor) execute(ctx context.Context, h handler.Node, nCtx *nodeEx nCtx.nsm.clearNodeStatus() } - // TODO @hamersaw - complete - if phase.GetPhase() == handler.EPhaseSuccess { - if cacheHandler, ok := h.(handler.CacheableNode); ok { - cacheable, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) - if err != nil { - logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) - return handler.PhaseInfoUndefined, err - } - - if cacheable { - status, err := c.WriteCatalogCache(ctx, nCtx, cacheHandler) - if err != nil { - // TODO @hamersaw fail - return handler.PhaseInfoUndefined, err - } - - // TODO @hamersaw - does this need to be outside? - // populate PhaseInfo with cache status - info := phase.GetInfo() - if info == nil { - info = &handler.ExecutionInfo{} - } - - if info.TaskNodeInfo == nil { - info.TaskNodeInfo = &handler.TaskNodeInfo{} - } - - if info.TaskNodeInfo.TaskNodeMetadata == nil { - info.TaskNodeInfo.TaskNodeMetadata = &event.TaskNodeMetadata{} - } - - info.TaskNodeInfo.TaskNodeMetadata.CacheStatus = status.GetCacheStatus() - info.TaskNodeInfo.TaskNodeMetadata.CatalogKey = status.GetMetadata() - phase = phase.WithInfo(info) - } - - if cacheSerializable { - _, err := c.ReleaseCatalogReservation(ctx, nCtx, cacheHandler) - if err != nil { - // TODO @hamersaw fail - return handler.PhaseInfoUndefined, err - } - } - } - } - return phase, nil } @@ -542,8 +496,7 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusPending, nil } - // TODO @hamersaw - complete - cacheStatus := core.CatalogCacheStatus_CACHE_DISABLED + var cacheStatus *catalog.Status if cacheHandler, ok := h.(handler.CacheableNode); ok { cacheable, _, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { @@ -552,17 +505,14 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor } else if cacheable { entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) if err != nil { - // TODO @hamersaw fail + logger.Errorf(ctx, "failed to check the catalog cache with err '%s'", err.Error()) return executors.NodeStatusUndefined, err } - cacheStatus = entry.GetStatus().GetCacheStatus() - if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { - // update NodeStatus to Success - nodeStatus.ClearSubNodeStatus() - nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) - - // set phaseInfo transition to include ... TODO @hamersaw + status := entry.GetStatus() + cacheStatus = &status + if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + // if cache hit we immediately transition the node to successful outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ OutputInfo: &handler.OutputInfo { @@ -610,8 +560,8 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusQueued, nil } else if np == v1alpha1.NodePhaseSkipped { return executors.NodeStatusSuccess, nil - } else if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { - // process downstream nodes + } else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + // if cache hit then we immediately process downstream nodes return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } @@ -627,11 +577,10 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut logger.Debugf(ctx, "node executing, current phase [%s]", currentPhase) defer logger.Debugf(ctx, "node execution completed") - // TODO @hamersaw - check cache serialize - cacheStatus := core.CatalogCacheStatus_CACHE_DISABLED - catalogReservationStatus := core.CatalogReservation_RESERVATION_DISABLED + var cacheStatus *catalog.Status + var catalogReservationStatus *core.CatalogReservation_Status if cacheHandler, ok := h.(handler.CacheableNode); ok { - cacheable, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) + _, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) return executors.NodeStatusUndefined, err @@ -639,20 +588,17 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut // TODO @hamersaw - document since we already check cache in queued the first time we hit this we shouldn't check it // could potentially use nodeStatus.GetMessage() check and update on RESERVATION_EXISTS - if cacheable && currentPhase == v1alpha1.NodePhaseQueued { + if cacheSerializable && currentPhase == v1alpha1.NodePhaseQueued { entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) if err != nil { - // TODO @hamersaw fail + logger.Errorf(ctx, "failed to check the catalog cache with err '%s'", err.Error()) return executors.NodeStatusUndefined, err } - cacheStatus = entry.GetStatus().GetCacheStatus() - if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { - // update NodeStatus to Success // TODO @hamersaw - validate that we don't need to do - //nodeStatus.ClearSubNodeStatus() - //nodeStatus.UpdatePhase(v1alpha1.NodePhaseSucceeded, v1.Now(), "completed successfully", nil) - - // set phaseInfo transition to include ... TODO @hamersaw + status := entry.GetStatus() + cacheStatus = &status + if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + // if cache hit we immediately transition the node to successful outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ OutputInfo: &handler.OutputInfo { @@ -668,39 +614,19 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } } - if cacheSerializable && cacheStatus != core.CatalogCacheStatus_CACHE_HIT { + if cacheSerializable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { entry, err := c.GetOrExtendCatalogReservation(ctx, nCtx, cacheHandler, config.GetConfig().WorkflowReEval.Duration) if err != nil { - // TODO @hamersaw fail + logger.Errorf(ctx, "failed to check for catalog reservation with err '%s'", err.Error()) return executors.NodeStatusUndefined, err } - // TODO @hamersaw - update phase info with reservation info - //p.execInfo.TaskNodeInfo.TaskNodeMetadata.ReservationStatus = entry.GetStatus() - //pluginTrns.PopulateReservationInfo(reservation) - - /*if reservation.GetStatus() == core.CatalogReservation_RESERVATION_ACQUIRED && - (ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache) { - logger.Infof(ctx, "Acquired cache reservation") - }*/ - - // If we do not own the reservation then we transition to WaitingForCache phase. If we are - // already running (ie. in a phase other than PhaseUndefined or PhaseWaitingForCache) and - // somehow lost the reservation (ex. by expiration), continue to execute until completion. - catalogReservationStatus = entry.GetStatus() - if catalogReservationStatus == core.CatalogReservation_RESERVATION_EXISTS { - /*if ts.PluginPhase == pluginCore.PhaseUndefined || ts.PluginPhase == pluginCore.PhaseWaitingForCache { - pluginTrns.ttype = handler.TransitionTypeEphemeral - pluginTrns.pInfo = pluginCore.PhaseInfoWaitingForCache(pluginCore.DefaultPhaseVersion, nil) - }*/ - - /*if currentPhase == v1alpha1.NodePhaseQueued { - //if ts.PluginPhase == pluginCore.PhaseWaitingForCache { - logger.Debugf(ctx, "No state change for Task, previously observed same transition. Short circuiting.") - p = handler.PhaseInfoQueued("waiting on serialized cache") - nodeStatus.UpdatePhase(v1alpha1.NodePhaseQueued, v1.Now(), "waiting on serialized cache", nil) - }*/ - + status := entry.GetStatus() + catalogReservationStatus = &status + if status == core.CatalogReservation_RESERVATION_ACQUIRED && currentPhase == v1alpha1.NodePhaseQueued { + logger.Infof(ctx, "acquired cache reservation") + } else if status == core.CatalogReservation_RESERVATION_EXISTS { + // if reservation is held by another owner we stay in the queued phase p = handler.PhaseInfoQueued("waiting on serialized cache") } } @@ -710,12 +636,12 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut // across execute which is used to emit metrics lastAttemptStartTime := nodeStatus.GetLastAttemptStartedAt() - // TODO @hamersaw - document - // a few scenarios we need to cover - // - cache hit - // - waiting on cache serialize - // - lost the cache reservation, but already running - should not happen. but should still progress - if currentPhase != v1alpha1.NodePhaseQueued || (cacheStatus != core.CatalogCacheStatus_CACHE_HIT && catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS) { + // we execute the node if: + // (1) caching is disabled (ie. cacheStatus == nil) + // (2) there was no cache hit and the cache is not blocked by a cache reservation + // (3) the node is already running, this covers the scenario where the node held the cache + // reservation, but it expired and was captured by a different node + if currentPhase != v1alpha1.NodePhaseQueued || ((cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) && (catalogReservationStatus == nil || *catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS)) { var err error p, err = c.execute(ctx, h, nCtx, nodeStatus) @@ -729,6 +655,38 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut return executors.NodeStatusUndefined, errors.Errorf(errors.IllegalStateError, nCtx.NodeID(), "received undefined phase.") } + if p.GetPhase() == handler.EPhaseSuccess { + if cacheHandler, ok := h.(handler.CacheableNode); ok { + // if node is cacheable we attempt to write outputs to the cache and release catalog reservation + cacheable, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return executors.NodeStatusUndefined, err + } + + if cacheable { + status, err := c.WriteCatalogCache(ctx, nCtx, cacheHandler) + if err != nil { + logger.Errorf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) + return executors.NodeStatusUndefined, err + } + + cacheStatus = &status + } + + if cacheSerializable { + entry, err := c.ReleaseCatalogReservation(ctx, nCtx, cacheHandler) + if err != nil { + // ignore failure to release the catalog reservation + logger.Warnf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) + } else { + status := entry.GetStatus() + catalogReservationStatus = &status + } + } + } + } + np, err := ToNodePhase(p.GetPhase()) if err != nil { return executors.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "failed to move from queued") @@ -841,13 +799,15 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } } + + p = updatePhaseCacheInfo(p, cacheStatus, catalogReservationStatus) UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) - if cacheStatus == core.CatalogCacheStatus_CACHE_HIT { + + if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { // process downstream nodes return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } - logger.Infof(ctx, "HAMERSAW end - %v %v", currentPhase, nodeStatus.GetMessage()) return finalStatus, nil } From 63109020a38f335fb6632b0506f8758f4413f129 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Sun, 9 Oct 2022 20:54:18 -0500 Subject: [PATCH 13/40] minor refactoring and bug fixes Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 2 ++ pkg/controller/nodes/executor.go | 13 ++++++++----- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index a778d6b58..c0dc5d3c5 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -42,6 +42,8 @@ func computeCatalogReservationOwnerID(ctx context.Context, nCtx *nodeExecContext return ownerID, nil } +// updatePhaseCacheInfo adds the cache and catalog reservation metadata to the PhaseInfo. This +// ensures this information is reported in events and available within FlyteAdmin. func updatePhaseCacheInfo(phaseInfo handler.PhaseInfo, cacheStatus *catalog.Status, reservationStatus *core.CatalogReservation_Status) handler.PhaseInfo { if cacheStatus == nil && reservationStatus == nil { return phaseInfo diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index d3b764897..867a1e189 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -664,17 +664,17 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut return executors.NodeStatusUndefined, err } - if cacheable { + if cacheable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { status, err := c.WriteCatalogCache(ctx, nCtx, cacheHandler) if err != nil { - logger.Errorf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) - return executors.NodeStatusUndefined, err + // ignore failure to write to catalog + logger.Warnf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) } cacheStatus = &status } - if cacheSerializable { + if cacheSerializable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { entry, err := c.ReleaseCatalogReservation(ctx, nCtx, cacheHandler) if err != nil { // ignore failure to release the catalog reservation @@ -687,6 +687,10 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } } + // update phase info with catalog cache and reservation information while maintaining all + // other metadata + p = updatePhaseCacheInfo(p, cacheStatus, catalogReservationStatus) + np, err := ToNodePhase(p.GetPhase()) if err != nil { return executors.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "failed to move from queued") @@ -800,7 +804,6 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } - p = updatePhaseCacheInfo(p, cacheStatus, catalogReservationStatus) UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { From 502d749b10baf7df5c58648b8712e2eae901dd3b Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 10 Oct 2022 10:49:06 -0500 Subject: [PATCH 14/40] doing an outputs lookup on cache to ensure correctness during failures Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 16 ++++++++- pkg/controller/nodes/executor.go | 57 +++++++++++++++++--------------- 2 files changed, 45 insertions(+), 28 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index c0dc5d3c5..a71821414 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -77,6 +77,21 @@ func updatePhaseCacheInfo(phaseInfo handler.PhaseInfo, cacheStatus *catalog.Stat // CheckCatalogCache uses the handler and contexts to check if cached outputs for the current node // exist. If the exist, this function also copies the outputs to this node. func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { + // Check if the outputs file already exists. Since processing downstream nodes is performed + // immediately following a cache hit, any failures will result in performing this operation + // again during the following node evaluation. In the scenario that outputs were already + // written we report a cache hit to ensure correctness in node evaluations. + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + metadata, err := nCtx.DataStore().Head(ctx, outputFile) + if err != nil { + logger.Errorf(ctx, "Failed to check the existence of outputs file. err: '%v'", err) + return catalog.Entry{}, errors.Wrapf(err, "failed to check the existence of deck file.") + } + + if metadata.Exists() { + return catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil)), nil + } + catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { return catalog.Entry{}, errors.Wrapf(err, "failed to initialize the catalogKey") @@ -118,7 +133,6 @@ func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecCont return catalog.Entry{}, nodeserrors.Errorf(nodeserrors.IllegalStateError, nCtx.NodeID(), "execution error from a cache output, bad state: %s", ee.String()) } - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) if err := nCtx.DataStore().WriteProtobuf(ctx, outputFile, storage.Options{}, o); err != nil { logger.Errorf(ctx, "failed to write cached value to datastore, err: %s", err.Error()) return catalog.Entry{}, err diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 867a1e189..eae967711 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -579,7 +579,10 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut var cacheStatus *catalog.Status var catalogReservationStatus *core.CatalogReservation_Status - if cacheHandler, ok := h.(handler.CacheableNode); ok { + cacheHandler, cacheHandlerOk := h.(handler.CacheableNode) + if cacheHandlerOk { + // if node is cacheable we attempt to check the cache if in queued phase or get / extend a + // catalog reservation _, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) if err != nil { logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) @@ -641,9 +644,11 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut // (2) there was no cache hit and the cache is not blocked by a cache reservation // (3) the node is already running, this covers the scenario where the node held the cache // reservation, but it expired and was captured by a different node - if currentPhase != v1alpha1.NodePhaseQueued || ((cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) && (catalogReservationStatus == nil || *catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS)) { - var err error + if currentPhase != v1alpha1.NodePhaseQueued || + ((cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) && + (catalogReservationStatus == nil || *catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS)) { + var err error p, err = c.execute(ctx, h, nCtx, nodeStatus) if err != nil { logger.Errorf(ctx, "failed Execute for node. Error: %s", err.Error()) @@ -655,34 +660,32 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut return executors.NodeStatusUndefined, errors.Errorf(errors.IllegalStateError, nCtx.NodeID(), "received undefined phase.") } - if p.GetPhase() == handler.EPhaseSuccess { - if cacheHandler, ok := h.(handler.CacheableNode); ok { - // if node is cacheable we attempt to write outputs to the cache and release catalog reservation - cacheable, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) + if p.GetPhase() == handler.EPhaseSuccess && cacheHandlerOk { + // if node is cacheable we attempt to write outputs to the cache and release catalog reservation + cacheable, cacheSerializable, err := cacheHandler.IsCacheable(ctx, nCtx) + if err != nil { + logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) + return executors.NodeStatusUndefined, err + } + + if cacheable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { + status, err := c.WriteCatalogCache(ctx, nCtx, cacheHandler) if err != nil { - logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) - return executors.NodeStatusUndefined, err + // ignore failure to write to catalog + logger.Warnf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) } - if cacheable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { - status, err := c.WriteCatalogCache(ctx, nCtx, cacheHandler) - if err != nil { - // ignore failure to write to catalog - logger.Warnf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) - } - - cacheStatus = &status - } + cacheStatus = &status + } - if cacheSerializable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { - entry, err := c.ReleaseCatalogReservation(ctx, nCtx, cacheHandler) - if err != nil { - // ignore failure to release the catalog reservation - logger.Warnf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) - } else { - status := entry.GetStatus() - catalogReservationStatus = &status - } + if cacheSerializable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { + entry, err := c.ReleaseCatalogReservation(ctx, nCtx, cacheHandler) + if err != nil { + // ignore failure to release the catalog reservation + logger.Warnf(ctx, "failed to write to the catalog cache with err '%s'", err.Error()) + } else { + status := entry.GetStatus() + catalogReservationStatus = &status } } } From fe2d056d68478b7d44de1bd0cf925f1c448ba457 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 10 Oct 2022 23:24:47 -0500 Subject: [PATCH 15/40] fix unit tests Signed-off-by: Daniel Rammer --- pkg/controller/nodes/dynamic/handler_test.go | 9 +- .../nodes/dynamic/mocks/task_node_handler.go | 136 +++-- .../nodes/handler/mocks/cacheable_node.go | 270 ++++++++++ pkg/controller/nodes/task/handler_test.go | 478 +----------------- 4 files changed, 386 insertions(+), 507 deletions(-) create mode 100644 pkg/controller/nodes/handler/mocks/cacheable_node.go diff --git a/pkg/controller/nodes/dynamic/handler_test.go b/pkg/controller/nodes/dynamic/handler_test.go index 6c048ec17..60e2ac2ca 100644 --- a/pkg/controller/nodes/dynamic/handler_test.go +++ b/pkg/controller/nodes/dynamic/handler_test.go @@ -7,7 +7,6 @@ import ( "github.com/flyteorg/flytepropeller/pkg/controller/config" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" "github.com/flyteorg/flytestdlib/contextutils" "github.com/flyteorg/flytestdlib/promutils/labeled" @@ -550,9 +549,9 @@ func Test_dynamicNodeHandler_Handle_SubTaskV1(t *testing.T) { mockLPLauncher := &lpMocks.Reader{} h := &mocks.TaskNodeHandler{} if tt.args.validErr != nil { - h.OnValidateOutputAndCacheAddMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), tt.args.validErr, nil) + h.OnValidateOutputMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(tt.args.validErr, nil) } else { - h.OnValidateOutputAndCacheAddMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, &core.CatalogMetadata{ArtifactTag: &core.CatalogArtifactTag{Name: "name", ArtifactId: "id"}}), nil, nil) + h.OnValidateOutputMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, nil) } n := &executorMocks.Node{} if tt.args.isErr { @@ -740,9 +739,9 @@ func Test_dynamicNodeHandler_Handle_SubTask(t *testing.T) { mockLPLauncher := &lpMocks.Reader{} h := &mocks.TaskNodeHandler{} if tt.args.validErr != nil { - h.OnValidateOutputAndCacheAddMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), tt.args.validErr, nil) + h.OnValidateOutputMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(tt.args.validErr, nil) } else { - h.OnValidateOutputAndCacheAddMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, &core.CatalogMetadata{ArtifactTag: &core.CatalogArtifactTag{Name: "name", ArtifactId: "id"}}), nil, nil) + h.OnValidateOutputMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, nil) } n := &executorMocks.Node{} if tt.args.isErr { diff --git a/pkg/controller/nodes/dynamic/mocks/task_node_handler.go b/pkg/controller/nodes/dynamic/mocks/task_node_handler.go index 49936c11d..29d3d0ecb 100644 --- a/pkg/controller/nodes/dynamic/mocks/task_node_handler.go +++ b/pkg/controller/nodes/dynamic/mocks/task_node_handler.go @@ -119,6 +119,45 @@ func (_m *TaskNodeHandler) FinalizeRequired() bool { return r0 } +type TaskNodeHandler_GetCatalogKey struct { + *mock.Call +} + +func (_m TaskNodeHandler_GetCatalogKey) Return(_a0 catalog.Key, _a1 error) *TaskNodeHandler_GetCatalogKey { + return &TaskNodeHandler_GetCatalogKey{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *TaskNodeHandler) OnGetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) *TaskNodeHandler_GetCatalogKey { + c_call := _m.On("GetCatalogKey", ctx, executionContext) + return &TaskNodeHandler_GetCatalogKey{Call: c_call} +} + +func (_m *TaskNodeHandler) OnGetCatalogKeyMatch(matchers ...interface{}) *TaskNodeHandler_GetCatalogKey { + c_call := _m.On("GetCatalogKey", matchers...) + return &TaskNodeHandler_GetCatalogKey{Call: c_call} +} + +// GetCatalogKey provides a mock function with given fields: ctx, executionContext +func (_m *TaskNodeHandler) GetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) (catalog.Key, error) { + ret := _m.Called(ctx, executionContext) + + var r0 catalog.Key + if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) catalog.Key); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(catalog.Key) + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) error); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + type TaskNodeHandler_Handle struct { *mock.Call } @@ -158,6 +197,52 @@ func (_m *TaskNodeHandler) Handle(ctx context.Context, executionContext handler. return r0, r1 } +type TaskNodeHandler_IsCacheable struct { + *mock.Call +} + +func (_m TaskNodeHandler_IsCacheable) Return(_a0 bool, _a1 bool, _a2 error) *TaskNodeHandler_IsCacheable { + return &TaskNodeHandler_IsCacheable{Call: _m.Call.Return(_a0, _a1, _a2)} +} + +func (_m *TaskNodeHandler) OnIsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) *TaskNodeHandler_IsCacheable { + c_call := _m.On("IsCacheable", ctx, executionContext) + return &TaskNodeHandler_IsCacheable{Call: c_call} +} + +func (_m *TaskNodeHandler) OnIsCacheableMatch(matchers ...interface{}) *TaskNodeHandler_IsCacheable { + c_call := _m.On("IsCacheable", matchers...) + return &TaskNodeHandler_IsCacheable{Call: c_call} +} + +// IsCacheable provides a mock function with given fields: ctx, executionContext +func (_m *TaskNodeHandler) IsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) (bool, bool, error) { + ret := _m.Called(ctx, executionContext) + + var r0 bool + if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) bool); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(bool) + } + + var r1 bool + if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) bool); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Get(1).(bool) + } + + var r2 error + if rf, ok := ret.Get(2).(func(context.Context, handler.NodeExecutionContext) error); ok { + r2 = rf(ctx, executionContext) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 +} + type TaskNodeHandler_Setup struct { *mock.Call } @@ -190,50 +275,43 @@ func (_m *TaskNodeHandler) Setup(ctx context.Context, setupContext handler.Setup return r0 } -type TaskNodeHandler_ValidateOutputAndCacheAdd struct { +type TaskNodeHandler_ValidateOutput struct { *mock.Call } -func (_m TaskNodeHandler_ValidateOutputAndCacheAdd) Return(_a0 catalog.Status, _a1 *io.ExecutionError, _a2 error) *TaskNodeHandler_ValidateOutputAndCacheAdd { - return &TaskNodeHandler_ValidateOutputAndCacheAdd{Call: _m.Call.Return(_a0, _a1, _a2)} +func (_m TaskNodeHandler_ValidateOutput) Return(_a0 *io.ExecutionError, _a1 error) *TaskNodeHandler_ValidateOutput { + return &TaskNodeHandler_ValidateOutput{Call: _m.Call.Return(_a0, _a1)} } -func (_m *TaskNodeHandler) OnValidateOutputAndCacheAdd(ctx context.Context, nodeID string, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader, m catalog.Metadata) *TaskNodeHandler_ValidateOutputAndCacheAdd { - c_call := _m.On("ValidateOutputAndCacheAdd", ctx, nodeID, i, r, outputCommitter, executionConfig, tr, m) - return &TaskNodeHandler_ValidateOutputAndCacheAdd{Call: c_call} +func (_m *TaskNodeHandler) OnValidateOutput(ctx context.Context, nodeID string, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader) *TaskNodeHandler_ValidateOutput { + c_call := _m.On("ValidateOutput", ctx, nodeID, i, r, outputCommitter, executionConfig, tr) + return &TaskNodeHandler_ValidateOutput{Call: c_call} } -func (_m *TaskNodeHandler) OnValidateOutputAndCacheAddMatch(matchers ...interface{}) *TaskNodeHandler_ValidateOutputAndCacheAdd { - c_call := _m.On("ValidateOutputAndCacheAdd", matchers...) - return &TaskNodeHandler_ValidateOutputAndCacheAdd{Call: c_call} +func (_m *TaskNodeHandler) OnValidateOutputMatch(matchers ...interface{}) *TaskNodeHandler_ValidateOutput { + c_call := _m.On("ValidateOutput", matchers...) + return &TaskNodeHandler_ValidateOutput{Call: c_call} } -// ValidateOutputAndCacheAdd provides a mock function with given fields: ctx, nodeID, i, r, outputCommitter, executionConfig, tr, m -func (_m *TaskNodeHandler) ValidateOutputAndCacheAdd(ctx context.Context, nodeID string, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader, m catalog.Metadata) (catalog.Status, *io.ExecutionError, error) { - ret := _m.Called(ctx, nodeID, i, r, outputCommitter, executionConfig, tr, m) +// ValidateOutput provides a mock function with given fields: ctx, nodeID, i, r, outputCommitter, executionConfig, tr +func (_m *TaskNodeHandler) ValidateOutput(ctx context.Context, nodeID string, i io.InputReader, r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, tr ioutils.SimpleTaskReader) (*io.ExecutionError, error) { + ret := _m.Called(ctx, nodeID, i, r, outputCommitter, executionConfig, tr) - var r0 catalog.Status - if rf, ok := ret.Get(0).(func(context.Context, string, io.InputReader, io.OutputReader, io.OutputWriter, v1alpha1.ExecutionConfig, ioutils.SimpleTaskReader, catalog.Metadata) catalog.Status); ok { - r0 = rf(ctx, nodeID, i, r, outputCommitter, executionConfig, tr, m) + var r0 *io.ExecutionError + if rf, ok := ret.Get(0).(func(context.Context, string, io.InputReader, io.OutputReader, io.OutputWriter, v1alpha1.ExecutionConfig, ioutils.SimpleTaskReader) *io.ExecutionError); ok { + r0 = rf(ctx, nodeID, i, r, outputCommitter, executionConfig, tr) } else { - r0 = ret.Get(0).(catalog.Status) - } - - var r1 *io.ExecutionError - if rf, ok := ret.Get(1).(func(context.Context, string, io.InputReader, io.OutputReader, io.OutputWriter, v1alpha1.ExecutionConfig, ioutils.SimpleTaskReader, catalog.Metadata) *io.ExecutionError); ok { - r1 = rf(ctx, nodeID, i, r, outputCommitter, executionConfig, tr, m) - } else { - if ret.Get(1) != nil { - r1 = ret.Get(1).(*io.ExecutionError) + if ret.Get(0) != nil { + r0 = ret.Get(0).(*io.ExecutionError) } } - var r2 error - if rf, ok := ret.Get(2).(func(context.Context, string, io.InputReader, io.OutputReader, io.OutputWriter, v1alpha1.ExecutionConfig, ioutils.SimpleTaskReader, catalog.Metadata) error); ok { - r2 = rf(ctx, nodeID, i, r, outputCommitter, executionConfig, tr, m) + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, io.InputReader, io.OutputReader, io.OutputWriter, v1alpha1.ExecutionConfig, ioutils.SimpleTaskReader) error); ok { + r1 = rf(ctx, nodeID, i, r, outputCommitter, executionConfig, tr) } else { - r2 = ret.Error(2) + r1 = ret.Error(1) } - return r0, r1, r2 + return r0, r1 } diff --git a/pkg/controller/nodes/handler/mocks/cacheable_node.go b/pkg/controller/nodes/handler/mocks/cacheable_node.go new file mode 100644 index 000000000..5e5ba50a7 --- /dev/null +++ b/pkg/controller/nodes/handler/mocks/cacheable_node.go @@ -0,0 +1,270 @@ +// Code generated by mockery v1.0.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + catalog "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + + handler "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" + + mock "github.com/stretchr/testify/mock" +) + +// CacheableNode is an autogenerated mock type for the CacheableNode type +type CacheableNode struct { + mock.Mock +} + +type CacheableNode_Abort struct { + *mock.Call +} + +func (_m CacheableNode_Abort) Return(_a0 error) *CacheableNode_Abort { + return &CacheableNode_Abort{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNode) OnAbort(ctx context.Context, executionContext handler.NodeExecutionContext, reason string) *CacheableNode_Abort { + c_call := _m.On("Abort", ctx, executionContext, reason) + return &CacheableNode_Abort{Call: c_call} +} + +func (_m *CacheableNode) OnAbortMatch(matchers ...interface{}) *CacheableNode_Abort { + c_call := _m.On("Abort", matchers...) + return &CacheableNode_Abort{Call: c_call} +} + +// Abort provides a mock function with given fields: ctx, executionContext, reason +func (_m *CacheableNode) Abort(ctx context.Context, executionContext handler.NodeExecutionContext, reason string) error { + ret := _m.Called(ctx, executionContext, reason) + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext, string) error); ok { + r0 = rf(ctx, executionContext, reason) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +type CacheableNode_Finalize struct { + *mock.Call +} + +func (_m CacheableNode_Finalize) Return(_a0 error) *CacheableNode_Finalize { + return &CacheableNode_Finalize{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNode) OnFinalize(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_Finalize { + c_call := _m.On("Finalize", ctx, executionContext) + return &CacheableNode_Finalize{Call: c_call} +} + +func (_m *CacheableNode) OnFinalizeMatch(matchers ...interface{}) *CacheableNode_Finalize { + c_call := _m.On("Finalize", matchers...) + return &CacheableNode_Finalize{Call: c_call} +} + +// Finalize provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNode) Finalize(ctx context.Context, executionContext handler.NodeExecutionContext) error { + ret := _m.Called(ctx, executionContext) + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) error); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +type CacheableNode_FinalizeRequired struct { + *mock.Call +} + +func (_m CacheableNode_FinalizeRequired) Return(_a0 bool) *CacheableNode_FinalizeRequired { + return &CacheableNode_FinalizeRequired{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNode) OnFinalizeRequired() *CacheableNode_FinalizeRequired { + c_call := _m.On("FinalizeRequired") + return &CacheableNode_FinalizeRequired{Call: c_call} +} + +func (_m *CacheableNode) OnFinalizeRequiredMatch(matchers ...interface{}) *CacheableNode_FinalizeRequired { + c_call := _m.On("FinalizeRequired", matchers...) + return &CacheableNode_FinalizeRequired{Call: c_call} +} + +// FinalizeRequired provides a mock function with given fields: +func (_m *CacheableNode) FinalizeRequired() bool { + ret := _m.Called() + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +type CacheableNode_GetCatalogKey struct { + *mock.Call +} + +func (_m CacheableNode_GetCatalogKey) Return(_a0 catalog.Key, _a1 error) *CacheableNode_GetCatalogKey { + return &CacheableNode_GetCatalogKey{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *CacheableNode) OnGetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_GetCatalogKey { + c_call := _m.On("GetCatalogKey", ctx, executionContext) + return &CacheableNode_GetCatalogKey{Call: c_call} +} + +func (_m *CacheableNode) OnGetCatalogKeyMatch(matchers ...interface{}) *CacheableNode_GetCatalogKey { + c_call := _m.On("GetCatalogKey", matchers...) + return &CacheableNode_GetCatalogKey{Call: c_call} +} + +// GetCatalogKey provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNode) GetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) (catalog.Key, error) { + ret := _m.Called(ctx, executionContext) + + var r0 catalog.Key + if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) catalog.Key); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(catalog.Key) + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) error); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type CacheableNode_Handle struct { + *mock.Call +} + +func (_m CacheableNode_Handle) Return(_a0 handler.Transition, _a1 error) *CacheableNode_Handle { + return &CacheableNode_Handle{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *CacheableNode) OnHandle(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_Handle { + c_call := _m.On("Handle", ctx, executionContext) + return &CacheableNode_Handle{Call: c_call} +} + +func (_m *CacheableNode) OnHandleMatch(matchers ...interface{}) *CacheableNode_Handle { + c_call := _m.On("Handle", matchers...) + return &CacheableNode_Handle{Call: c_call} +} + +// Handle provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNode) Handle(ctx context.Context, executionContext handler.NodeExecutionContext) (handler.Transition, error) { + ret := _m.Called(ctx, executionContext) + + var r0 handler.Transition + if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) handler.Transition); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(handler.Transition) + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) error); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type CacheableNode_IsCacheable struct { + *mock.Call +} + +func (_m CacheableNode_IsCacheable) Return(_a0 bool, _a1 bool, _a2 error) *CacheableNode_IsCacheable { + return &CacheableNode_IsCacheable{Call: _m.Call.Return(_a0, _a1, _a2)} +} + +func (_m *CacheableNode) OnIsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_IsCacheable { + c_call := _m.On("IsCacheable", ctx, executionContext) + return &CacheableNode_IsCacheable{Call: c_call} +} + +func (_m *CacheableNode) OnIsCacheableMatch(matchers ...interface{}) *CacheableNode_IsCacheable { + c_call := _m.On("IsCacheable", matchers...) + return &CacheableNode_IsCacheable{Call: c_call} +} + +// IsCacheable provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNode) IsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) (bool, bool, error) { + ret := _m.Called(ctx, executionContext) + + var r0 bool + if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) bool); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(bool) + } + + var r1 bool + if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) bool); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Get(1).(bool) + } + + var r2 error + if rf, ok := ret.Get(2).(func(context.Context, handler.NodeExecutionContext) error); ok { + r2 = rf(ctx, executionContext) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 +} + +type CacheableNode_Setup struct { + *mock.Call +} + +func (_m CacheableNode_Setup) Return(_a0 error) *CacheableNode_Setup { + return &CacheableNode_Setup{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNode) OnSetup(ctx context.Context, setupContext handler.SetupContext) *CacheableNode_Setup { + c_call := _m.On("Setup", ctx, setupContext) + return &CacheableNode_Setup{Call: c_call} +} + +func (_m *CacheableNode) OnSetupMatch(matchers ...interface{}) *CacheableNode_Setup { + c_call := _m.On("Setup", matchers...) + return &CacheableNode_Setup{Call: c_call} +} + +// Setup provides a mock function with given fields: ctx, setupContext +func (_m *CacheableNode) Setup(ctx context.Context, setupContext handler.SetupContext) error { + ret := _m.Called(ctx, setupContext) + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, handler.SetupContext) error); ok { + r0 = rf(ctx, setupContext) + } else { + r0 = ret.Error(0) + } + + return r0 +} diff --git a/pkg/controller/nodes/task/handler_test.go b/pkg/controller/nodes/task/handler_test.go index bc2487a6a..a6cf3880a 100644 --- a/pkg/controller/nodes/task/handler_test.go +++ b/pkg/controller/nodes/task/handler_test.go @@ -17,7 +17,6 @@ import ( "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/admin" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task/resourcemanager" @@ -26,7 +25,6 @@ import ( "github.com/flyteorg/flytestdlib/promutils/labeled" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" - "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/datacatalog" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery" pluginCatalogMocks "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog/mocks" @@ -732,442 +730,6 @@ func Test_task_Handle_NoCatalog(t *testing.T) { } } -func Test_task_Handle_Catalog(t *testing.T) { - - createNodeContext := func(recorder events.TaskEventRecorder, ttype string, s *taskNodeStateHolder) *nodeMocks.NodeExecutionContext { - wfExecID := &core.WorkflowExecutionIdentifier{ - Project: "project", - Domain: "domain", - Name: "name", - } - - nodeID := "n1" - - nm := &nodeMocks.NodeExecutionMetadata{} - nm.OnGetAnnotations().Return(map[string]string{}) - nm.OnGetNodeExecutionID().Return(&core.NodeExecutionIdentifier{ - NodeId: nodeID, - ExecutionId: wfExecID, - }) - nm.OnGetK8sServiceAccount().Return("service-account") - nm.OnGetLabels().Return(map[string]string{}) - nm.OnGetNamespace().Return("namespace") - nm.OnGetOwnerID().Return(types.NamespacedName{Namespace: "namespace", Name: "name"}) - nm.OnGetOwnerReference().Return(v12.OwnerReference{ - Kind: "sample", - Name: "name", - }) - nm.OnIsInterruptible().Return(true) - - taskID := &core.Identifier{} - tk := &core.TaskTemplate{ - Id: taskID, - Type: "test", - Metadata: &core.TaskMetadata{ - Discoverable: true, - }, - Interface: &core.TypedInterface{ - Outputs: &core.VariableMap{ - Variables: map[string]*core.Variable{ - "x": { - Type: &core.LiteralType{ - Type: &core.LiteralType_Simple{ - Simple: core.SimpleType_BOOLEAN, - }, - }, - }, - }, - }, - }, - } - tr := &nodeMocks.TaskReader{} - tr.OnGetTaskID().Return(taskID) - tr.OnGetTaskType().Return(ttype) - tr.OnReadMatch(mock.Anything).Return(tk, nil) - - ns := &flyteMocks.ExecutableNodeStatus{} - ns.OnGetDataDir().Return(storage.DataReference("data-dir")) - ns.OnGetOutputDir().Return(storage.DataReference("output-dir")) - - res := &v1.ResourceRequirements{} - n := &flyteMocks.ExecutableNode{} - ma := 5 - n.OnGetRetryStrategy().Return(&v1alpha1.RetryStrategy{MinAttempts: &ma}) - n.OnGetResources().Return(res) - - ir := &ioMocks.InputReader{} - ir.OnGetInputPath().Return(storage.DataReference("input")) - nCtx := &nodeMocks.NodeExecutionContext{} - nCtx.OnNodeExecutionMetadata().Return(nm) - nCtx.OnNode().Return(n) - nCtx.OnInputReader().Return(ir) - ds, err := storage.NewDataStore( - &storage.Config{ - Type: storage.TypeMemory, - }, - promutils.NewTestScope(), - ) - assert.NoError(t, err) - nCtx.OnDataStore().Return(ds) - nCtx.OnCurrentAttempt().Return(uint32(1)) - nCtx.OnTaskReader().Return(tr) - nCtx.OnMaxDatasetSizeBytes().Return(int64(1)) - nCtx.OnNodeStatus().Return(ns) - nCtx.OnNodeID().Return(nodeID) - nCtx.OnEventsRecorder().Return(recorder) - nCtx.OnEnqueueOwnerFunc().Return(nil) - - executionContext := &mocks.ExecutionContext{} - executionContext.OnGetExecutionConfig().Return(v1alpha1.ExecutionConfig{}) - executionContext.OnGetEventVersion().Return(v1alpha1.EventVersion0) - executionContext.OnGetParentInfo().Return(nil) - nCtx.OnExecutionContext().Return(executionContext) - - nCtx.OnRawOutputPrefix().Return("s3://sandbox/") - nCtx.OnOutputShardSelector().Return(ioutils.NewConstantShardSelector([]string{"x"})) - - st := bytes.NewBuffer([]byte{}) - cod := codex.GobStateCodec{} - assert.NoError(t, cod.Encode(&fakeplugins.NextPhaseState{ - Phase: pluginCore.PhaseSuccess, - OutputExists: true, - }, st)) - nr := &nodeMocks.NodeStateReader{} - nr.OnGetTaskNodeState().Return(handler.TaskNodeState{ - PluginState: st.Bytes(), - }) - nCtx.OnNodeStateReader().Return(nr) - nCtx.OnNodeStateWriter().Return(s) - return nCtx - } - - noopRm := CreateNoopResourceManager(context.TODO(), promutils.NewTestScope()) - - type args struct { - catalogFetch bool - catalogFetchError bool - catalogWriteError bool - } - type want struct { - handlerPhase handler.EPhase - wantErr bool - eventPhase core.TaskExecution_Phase - } - tests := []struct { - name string - args args - want want - }{ - { - "cache-hit", - args{ - catalogFetch: true, - catalogWriteError: true, - }, - want{ - handlerPhase: handler.EPhaseSuccess, - eventPhase: core.TaskExecution_SUCCEEDED, - }, - }, - { - "cache-err", - args{ - catalogFetchError: true, - catalogWriteError: true, - }, - want{ - handlerPhase: handler.EPhaseSuccess, - eventPhase: core.TaskExecution_SUCCEEDED, - }, - }, - { - "cache-write", - args{}, - want{ - handlerPhase: handler.EPhaseSuccess, - eventPhase: core.TaskExecution_SUCCEEDED, - }, - }, - { - "cache-write-err", - args{ - catalogWriteError: true, - }, - want{ - handlerPhase: handler.EPhaseSuccess, - eventPhase: core.TaskExecution_SUCCEEDED, - }, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - state := &taskNodeStateHolder{} - ev := &fakeBufferedTaskEventRecorder{} - nCtx := createNodeContext(ev, "test", state) - c := &pluginCatalogMocks.Client{} - if tt.args.catalogFetch { - or := &ioMocks.OutputReader{} - or.OnDeckExistsMatch(mock.Anything).Return(true, nil) - or.OnReadMatch(mock.Anything).Return(&core.LiteralMap{}, nil, nil) - c.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.NewCatalogEntry(or, catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil)), nil) - } else { - c.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.NewFailedCatalogEntry(catalog.NewStatus(core.CatalogCacheStatus_CACHE_MISS, nil)), nil) - } - if tt.args.catalogFetchError { - c.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.Entry{}, fmt.Errorf("failed to read from catalog")) - } - if tt.args.catalogWriteError { - c.OnPutMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(catalog.Status{}, fmt.Errorf("failed to write to catalog")) - } else { - c.OnPutMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(catalog.NewStatus(core.CatalogCacheStatus_CACHE_POPULATED, nil), nil) - } - tk, err := New(context.TODO(), mocks.NewFakeKubeClient(), c, eventConfig, testClusterID, promutils.NewTestScope()) - assert.NoError(t, err) - tk.defaultPlugins = map[pluginCore.TaskType]pluginCore.Plugin{ - "test": fakeplugins.NewPhaseBasedPlugin(), - } - tk.catalog = c - tk.resourceManager = noopRm - got, err := tk.Handle(context.TODO(), nCtx) - if (err != nil) != tt.want.wantErr { - t.Errorf("Handler.Handle() error = %v, wantErr %v", err, tt.want.wantErr) - return - } - if err == nil { - assert.Equal(t, tt.want.handlerPhase.String(), got.Info().GetPhase().String()) - if assert.Equal(t, 1, len(ev.evs)) { - e := ev.evs[0] - assert.Equal(t, tt.want.eventPhase.String(), e.Phase.String()) - } - assert.Equal(t, pluginCore.PhaseSuccess.String(), state.s.PluginPhase.String()) - assert.Equal(t, uint32(0), state.s.PluginPhaseVersion) - if tt.args.catalogFetch { - if assert.NotNil(t, got.Info().GetInfo().TaskNodeInfo) { - assert.NotNil(t, got.Info().GetInfo().TaskNodeInfo.TaskNodeMetadata) - assert.Equal(t, core.CatalogCacheStatus_CACHE_HIT, got.Info().GetInfo().TaskNodeInfo.TaskNodeMetadata.CacheStatus) - } - assert.NotNil(t, got.Info().GetInfo().OutputInfo) - s := storage.DataReference("/output-dir/outputs.pb") - assert.Equal(t, s, got.Info().GetInfo().OutputInfo.OutputURI) - r, err := nCtx.DataStore().Head(context.TODO(), s) - assert.NoError(t, err) - assert.True(t, r.Exists()) - } - } - }) - } -} - -func Test_task_Handle_Reservation(t *testing.T) { - - createNodeContext := func(recorder events.TaskEventRecorder, ttype string, s *taskNodeStateHolder) *nodeMocks.NodeExecutionContext { - wfExecID := &core.WorkflowExecutionIdentifier{ - Project: "project", - Domain: "domain", - Name: "name", - } - - nodeID := "n1" - - nm := &nodeMocks.NodeExecutionMetadata{} - nm.OnGetAnnotations().Return(map[string]string{}) - nm.OnGetNodeExecutionID().Return(&core.NodeExecutionIdentifier{ - NodeId: nodeID, - ExecutionId: wfExecID, - }) - nm.OnGetK8sServiceAccount().Return("service-account") - nm.OnGetLabels().Return(map[string]string{}) - nm.OnGetNamespace().Return("namespace") - nm.OnGetOwnerID().Return(types.NamespacedName{Namespace: "namespace", Name: "name"}) - nm.OnGetOwnerReference().Return(v12.OwnerReference{ - Kind: "sample", - Name: "name", - }) - nm.OnIsInterruptible().Return(true) - - taskID := &core.Identifier{} - tk := &core.TaskTemplate{ - Id: taskID, - Type: "test", - Metadata: &core.TaskMetadata{ - Discoverable: true, - CacheSerializable: true, - }, - Interface: &core.TypedInterface{ - Outputs: &core.VariableMap{ - Variables: map[string]*core.Variable{ - "x": { - Type: &core.LiteralType{ - Type: &core.LiteralType_Simple{ - Simple: core.SimpleType_BOOLEAN, - }, - }, - }, - }, - }, - }, - } - tr := &nodeMocks.TaskReader{} - tr.OnGetTaskID().Return(taskID) - tr.OnGetTaskType().Return(ttype) - tr.OnReadMatch(mock.Anything).Return(tk, nil) - - ns := &flyteMocks.ExecutableNodeStatus{} - ns.OnGetDataDir().Return(storage.DataReference("data-dir")) - ns.OnGetOutputDir().Return(storage.DataReference("output-dir")) - - res := &v1.ResourceRequirements{} - n := &flyteMocks.ExecutableNode{} - ma := 5 - n.OnGetRetryStrategy().Return(&v1alpha1.RetryStrategy{MinAttempts: &ma}) - n.OnGetResources().Return(res) - - ir := &ioMocks.InputReader{} - ir.OnGetInputPath().Return(storage.DataReference("input")) - nCtx := &nodeMocks.NodeExecutionContext{} - nCtx.OnNodeExecutionMetadata().Return(nm) - nCtx.OnNode().Return(n) - nCtx.OnInputReader().Return(ir) - nCtx.OnInputReader().Return(ir) - ds, err := storage.NewDataStore( - &storage.Config{ - Type: storage.TypeMemory, - }, - promutils.NewTestScope(), - ) - assert.NoError(t, err) - nCtx.OnDataStore().Return(ds) - nCtx.OnCurrentAttempt().Return(uint32(1)) - nCtx.OnTaskReader().Return(tr) - nCtx.OnMaxDatasetSizeBytes().Return(int64(1)) - nCtx.OnNodeStatus().Return(ns) - nCtx.OnNodeID().Return(nodeID) - nCtx.OnEventsRecorder().Return(recorder) - nCtx.OnEnqueueOwnerFunc().Return(nil) - - executionContext := &mocks.ExecutionContext{} - executionContext.OnGetExecutionConfig().Return(v1alpha1.ExecutionConfig{}) - executionContext.OnGetEventVersion().Return(v1alpha1.EventVersion0) - executionContext.OnGetParentInfo().Return(nil) - executionContext.OnIncrementParallelism().Return(1) - nCtx.OnExecutionContext().Return(executionContext) - - nCtx.OnRawOutputPrefix().Return("s3://sandbox/") - nCtx.OnOutputShardSelector().Return(ioutils.NewConstantShardSelector([]string{"x"})) - - nCtx.OnNodeStateWriter().Return(s) - return nCtx - } - - noopRm := CreateNoopResourceManager(context.TODO(), promutils.NewTestScope()) - - type args struct { - catalogFetch bool - pluginPhase pluginCore.Phase - ownerID string - } - type want struct { - pluginPhase pluginCore.Phase - handlerPhase handler.EPhase - eventPhase core.TaskExecution_Phase - } - tests := []struct { - name string - args args - want want - }{ - { - "reservation-create-or-update", - args{ - catalogFetch: false, - pluginPhase: pluginCore.PhaseUndefined, - ownerID: "name-n1-1", - }, - want{ - pluginPhase: pluginCore.PhaseSuccess, - handlerPhase: handler.EPhaseSuccess, - eventPhase: core.TaskExecution_SUCCEEDED, - }, - }, - { - "reservation-exists", - args{ - catalogFetch: false, - pluginPhase: pluginCore.PhaseUndefined, - ownerID: "nilOwner", - }, - want{ - pluginPhase: pluginCore.PhaseWaitingForCache, - handlerPhase: handler.EPhaseRunning, - eventPhase: core.TaskExecution_UNDEFINED, - }, - }, - { - "cache-hit", - args{ - catalogFetch: true, - pluginPhase: pluginCore.PhaseWaitingForCache, - }, - want{ - pluginPhase: pluginCore.PhaseSuccess, - handlerPhase: handler.EPhaseSuccess, - eventPhase: core.TaskExecution_SUCCEEDED, - }, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - state := &taskNodeStateHolder{} - ev := &fakeBufferedTaskEventRecorder{} - nCtx := createNodeContext(ev, "test", state) - c := &pluginCatalogMocks.Client{} - nr := &nodeMocks.NodeStateReader{} - st := bytes.NewBuffer([]byte{}) - cod := codex.GobStateCodec{} - assert.NoError(t, cod.Encode(&fakeplugins.NextPhaseState{ - Phase: pluginCore.PhaseSuccess, - OutputExists: true, - }, st)) - nr.OnGetTaskNodeState().Return(handler.TaskNodeState{ - PluginPhase: tt.args.pluginPhase, - PluginState: st.Bytes(), - }) - nCtx.OnNodeStateReader().Return(nr) - if tt.args.catalogFetch { - or := &ioMocks.OutputReader{} - or.OnDeckExistsMatch(mock.Anything).Return(true, nil) - or.OnReadMatch(mock.Anything).Return(&core.LiteralMap{}, nil, nil) - c.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.NewCatalogEntry(or, catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil)), nil) - } else { - c.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.NewFailedCatalogEntry(catalog.NewStatus(core.CatalogCacheStatus_CACHE_MISS, nil)), nil) - } - c.OnPutMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(catalog.NewStatus(core.CatalogCacheStatus_CACHE_POPULATED, nil), nil) - c.OnGetOrExtendReservationMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&datacatalog.Reservation{OwnerId: tt.args.ownerID}, nil) - tk, err := New(context.TODO(), mocks.NewFakeKubeClient(), c, eventConfig, testClusterID, promutils.NewTestScope()) - assert.NoError(t, err) - tk.defaultPlugins = map[pluginCore.TaskType]pluginCore.Plugin{ - "test": fakeplugins.NewPhaseBasedPlugin(), - } - tk.catalog = c - tk.resourceManager = noopRm - got, err := tk.Handle(context.TODO(), nCtx) - if err != nil { - t.Errorf("Handler.Handle() error = %v", err) - return - } - if err == nil { - assert.Equal(t, tt.want.handlerPhase.String(), got.Info().GetPhase().String()) - if assert.Equal(t, 1, len(ev.evs)) { - e := ev.evs[0] - assert.Equal(t, tt.want.eventPhase.String(), e.Phase.String()) - } - assert.Equal(t, tt.want.pluginPhase.String(), state.s.PluginPhase.String()) - assert.Equal(t, uint32(0), state.s.PluginPhaseVersion) - } - }) - } -} - func Test_task_Handle_Barrier(t *testing.T) { // NOTE: Caching is disabled for this test @@ -1797,7 +1359,7 @@ func Test_task_Abort_v1(t *testing.T) { func Test_task_Finalize(t *testing.T) { - createNodeContext := func(cacheSerializable bool) *nodeMocks.NodeExecutionContext { + createNodeContext := func() *nodeMocks.NodeExecutionContext { wfExecID := &core.WorkflowExecutionIdentifier{ Project: "project", Domain: "domain", @@ -1825,10 +1387,6 @@ func Test_task_Finalize(t *testing.T) { tk := &core.TaskTemplate{ Id: taskID, Type: "test", - Metadata: &core.TaskMetadata{ - CacheSerializable: cacheSerializable, - Discoverable: cacheSerializable, - }, Interface: &core.TypedInterface{ Outputs: &core.VariableMap{ Variables: map[string]*core.Variable{ @@ -1907,61 +1465,35 @@ func Test_task_Finalize(t *testing.T) { type fields struct { defaultPluginCallback func() pluginCore.Plugin } - type args struct { - releaseReservation bool - releaseReservationError bool - } tests := []struct { name string fields fields - args args wantErr bool finalize bool }{ {"no-plugin", fields{defaultPluginCallback: func() pluginCore.Plugin { return nil - }}, args{}, true, false}, - + }}, true, false}, {"finalize-fails", fields{defaultPluginCallback: func() pluginCore.Plugin { p := &pluginCoreMocks.Plugin{} p.On("GetID").Return("id") p.OnGetProperties().Return(pluginCore.PluginProperties{}) p.On("Finalize", mock.Anything, mock.Anything).Return(fmt.Errorf("error")) return p - }}, args{}, true, true}, + }}, true, true}, {"finalize-success", fields{defaultPluginCallback: func() pluginCore.Plugin { p := &pluginCoreMocks.Plugin{} p.On("GetID").Return("id") p.OnGetProperties().Return(pluginCore.PluginProperties{}) p.On("Finalize", mock.Anything, mock.Anything).Return(nil) return p - }}, args{}, false, true}, - {"release-reservation", fields{defaultPluginCallback: func() pluginCore.Plugin { - p := &pluginCoreMocks.Plugin{} - p.On("GetID").Return("id") - p.OnGetProperties().Return(pluginCore.PluginProperties{}) - p.On("Finalize", mock.Anything, mock.Anything).Return(nil) - return p - }}, args{releaseReservation: true}, false, true}, - {"release-reservation-error", fields{defaultPluginCallback: func() pluginCore.Plugin { - p := &pluginCoreMocks.Plugin{} - p.On("GetID").Return("id") - p.OnGetProperties().Return(pluginCore.PluginProperties{}) - p.On("Finalize", mock.Anything, mock.Anything).Return(nil) - return p - }}, args{releaseReservation: true, releaseReservationError: true}, true, false}, + }}, false, true}, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - nCtx := createNodeContext(tt.args.releaseReservation) + nCtx := createNodeContext() catalog := &pluginCatalogMocks.Client{} - if tt.args.releaseReservationError { - catalog.OnReleaseReservationMatch(mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("failed to release reservation")) - } else { - catalog.OnReleaseReservationMatch(mock.Anything, mock.Anything, mock.Anything).Return(nil) - } - m := tt.fields.defaultPluginCallback() tk, err := New(context.TODO(), mocks.NewFakeKubeClient(), catalog, eventConfig, testClusterID, promutils.NewTestScope()) assert.NoError(t, err) From a3e158130fa85fb01dbdb683858f2471479cdf86 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 10 Oct 2022 23:28:27 -0500 Subject: [PATCH 16/40] fixed lint issues Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 15 +++++++-------- pkg/controller/nodes/handler/iface.go | 2 +- pkg/controller/nodes/task/handler.go | 10 +++++----- pkg/controller/nodes/task/pre_post_execution.go | 14 +++++++------- 4 files changed, 20 insertions(+), 21 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index eae967711..66d673f5f 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -512,13 +512,13 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor status := entry.GetStatus() cacheStatus = &status if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - // if cache hit we immediately transition the node to successful + // if cache hit we immediately transition the node to successful outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ - OutputInfo: &handler.OutputInfo { + OutputInfo: &handler.OutputInfo{ OutputURI: outputFile, }, - TaskNodeInfo: &handler.TaskNodeInfo { + TaskNodeInfo: &handler.TaskNodeInfo{ TaskNodeMetadata: &event.TaskNodeMetadata{ CacheStatus: entry.GetStatus().GetCacheStatus(), CatalogKey: entry.GetStatus().GetMetadata(), @@ -601,13 +601,13 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut status := entry.GetStatus() cacheStatus = &status if entry.GetStatus().GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - // if cache hit we immediately transition the node to successful + // if cache hit we immediately transition the node to successful outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) p = handler.PhaseInfoSuccess(&handler.ExecutionInfo{ - OutputInfo: &handler.OutputInfo { + OutputInfo: &handler.OutputInfo{ OutputURI: outputFile, }, - TaskNodeInfo: &handler.TaskNodeInfo { + TaskNodeInfo: &handler.TaskNodeInfo{ TaskNodeMetadata: &event.TaskNodeMetadata{ CacheStatus: entry.GetStatus().GetCacheStatus(), CatalogKey: entry.GetStatus().GetMetadata(), @@ -646,7 +646,7 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut // reservation, but it expired and was captured by a different node if currentPhase != v1alpha1.NodePhaseQueued || ((cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) && - (catalogReservationStatus == nil || *catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS)) { + (catalogReservationStatus == nil || *catalogReservationStatus != core.CatalogReservation_RESERVATION_EXISTS)) { var err error p, err = c.execute(ctx, h, nCtx, nodeStatus) @@ -806,7 +806,6 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } } - UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { diff --git a/pkg/controller/nodes/handler/iface.go b/pkg/controller/nodes/handler/iface.go index 165147d8a..ea9b3175d 100644 --- a/pkg/controller/nodes/handler/iface.go +++ b/pkg/controller/nodes/handler/iface.go @@ -31,7 +31,7 @@ type Node interface { type CacheableNode interface { Node - // GetCatalogKey returns the unique key for the node represented by the NodeExecutionContext + // GetCatalogKey returns the unique key for the node represented by the NodeExecutionContext GetCatalogKey(ctx context.Context, executionContext NodeExecutionContext) (catalog.Key, error) // IsCacheable returns two booleans representing if the node represented by the diff --git a/pkg/controller/nodes/task/handler.go b/pkg/controller/nodes/task/handler.go index 99a2952f8..dca47d1d6 100644 --- a/pkg/controller/nodes/task/handler.go +++ b/pkg/controller/nodes/task/handler.go @@ -779,11 +779,11 @@ func New(ctx context.Context, kubeClient executors.Client, client catalog.Client pluginsForType: make(map[pluginCore.TaskType]map[pluginID]pluginCore.Plugin), taskMetricsMap: make(map[MetricKey]*taskMetrics), metrics: &metrics{ - pluginPanics: labeled.NewCounter("plugin_panic", "Task plugin paniced when trying to execute a Handler.", scope), - unsupportedTaskType: labeled.NewCounter("unsupported_tasktype", "No Handler plugin configured for Handler type", scope), - pluginExecutionLatency: labeled.NewStopWatch("plugin_exec_latency", "Time taken to invoke plugin for one round", time.Microsecond, scope), - pluginQueueLatency: labeled.NewStopWatch("plugin_queue_latency", "Time spent by plugin in queued phase", time.Microsecond, scope), - scope: scope, + pluginPanics: labeled.NewCounter("plugin_panic", "Task plugin paniced when trying to execute a Handler.", scope), + unsupportedTaskType: labeled.NewCounter("unsupported_tasktype", "No Handler plugin configured for Handler type", scope), + pluginExecutionLatency: labeled.NewStopWatch("plugin_exec_latency", "Time taken to invoke plugin for one round", time.Microsecond, scope), + pluginQueueLatency: labeled.NewStopWatch("plugin_queue_latency", "Time spent by plugin in queued phase", time.Microsecond, scope), + scope: scope, }, pluginScope: scope.NewSubScope("plugin"), kubeClient: kubeClient, diff --git a/pkg/controller/nodes/task/pre_post_execution.go b/pkg/controller/nodes/task/pre_post_execution.go index e7439c2c2..9d0c864e5 100644 --- a/pkg/controller/nodes/task/pre_post_execution.go +++ b/pkg/controller/nodes/task/pre_post_execution.go @@ -10,8 +10,8 @@ import ( "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" errors2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" "github.com/flyteorg/flytestdlib/contextutils" "github.com/flyteorg/flytestdlib/logger" @@ -127,12 +127,12 @@ func (t *Handler) ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i if !ok { // Does not exist return &io.ExecutionError{ - ExecutionError: &core.ExecutionError{ - Code: "OutputsNotFound", - Message: "Outputs not generated by task execution", - }, - IsRecoverable: true, - }, nil + ExecutionError: &core.ExecutionError{ + Code: "OutputsNotFound", + Message: "Outputs not generated by task execution", + }, + IsRecoverable: true, + }, nil } if !r.IsFile(ctx) { From 8a49895636c3692bc1ea340bdcfcfef477031ae4 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 10 Oct 2022 23:32:07 -0500 Subject: [PATCH 17/40] moved catalog package to the node level Signed-off-by: Daniel Rammer --- pkg/controller/controller.go | 2 +- pkg/controller/nodes/{task => }/catalog/config.go | 2 +- pkg/controller/nodes/{task => }/catalog/config_flags.go | 0 pkg/controller/nodes/{task => }/catalog/config_flags_test.go | 0 .../nodes/{task => }/catalog/datacatalog/datacatalog.go | 0 .../nodes/{task => }/catalog/datacatalog/datacatalog_test.go | 0 .../nodes/{task => }/catalog/datacatalog/transformer.go | 0 .../nodes/{task => }/catalog/datacatalog/transformer_test.go | 0 pkg/controller/nodes/{task => }/catalog/noop_catalog.go | 0 pkg/controller/nodes/executor.go | 4 ++-- 10 files changed, 4 insertions(+), 4 deletions(-) rename pkg/controller/nodes/{task => }/catalog/config.go (96%) rename pkg/controller/nodes/{task => }/catalog/config_flags.go (100%) rename pkg/controller/nodes/{task => }/catalog/config_flags_test.go (100%) rename pkg/controller/nodes/{task => }/catalog/datacatalog/datacatalog.go (100%) rename pkg/controller/nodes/{task => }/catalog/datacatalog/datacatalog_test.go (100%) rename pkg/controller/nodes/{task => }/catalog/datacatalog/transformer.go (100%) rename pkg/controller/nodes/{task => }/catalog/datacatalog/transformer_test.go (100%) rename pkg/controller/nodes/{task => }/catalog/noop_catalog.go (100%) diff --git a/pkg/controller/controller.go b/pkg/controller/controller.go index c536e019a..5dc8ce2d2 100644 --- a/pkg/controller/controller.go +++ b/pkg/controller/controller.go @@ -25,10 +25,10 @@ import ( "github.com/flyteorg/flytepropeller/pkg/controller/config" "github.com/flyteorg/flytepropeller/pkg/controller/executors" "github.com/flyteorg/flytepropeller/pkg/controller/nodes" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/catalog" errors3 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/recovery" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task/catalog" "github.com/flyteorg/flytepropeller/pkg/controller/workflow" "github.com/flyteorg/flytepropeller/pkg/controller/workflowstore" leader "github.com/flyteorg/flytepropeller/pkg/leaderelection" diff --git a/pkg/controller/nodes/task/catalog/config.go b/pkg/controller/nodes/catalog/config.go similarity index 96% rename from pkg/controller/nodes/task/catalog/config.go rename to pkg/controller/nodes/catalog/config.go index 20663cc1e..a77ba6f74 100644 --- a/pkg/controller/nodes/task/catalog/config.go +++ b/pkg/controller/nodes/catalog/config.go @@ -8,7 +8,7 @@ import ( "github.com/flyteorg/flytestdlib/config" "google.golang.org/grpc" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task/catalog/datacatalog" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/catalog/datacatalog" ) //go:generate pflags Config --default-var defaultConfig diff --git a/pkg/controller/nodes/task/catalog/config_flags.go b/pkg/controller/nodes/catalog/config_flags.go similarity index 100% rename from pkg/controller/nodes/task/catalog/config_flags.go rename to pkg/controller/nodes/catalog/config_flags.go diff --git a/pkg/controller/nodes/task/catalog/config_flags_test.go b/pkg/controller/nodes/catalog/config_flags_test.go similarity index 100% rename from pkg/controller/nodes/task/catalog/config_flags_test.go rename to pkg/controller/nodes/catalog/config_flags_test.go diff --git a/pkg/controller/nodes/task/catalog/datacatalog/datacatalog.go b/pkg/controller/nodes/catalog/datacatalog/datacatalog.go similarity index 100% rename from pkg/controller/nodes/task/catalog/datacatalog/datacatalog.go rename to pkg/controller/nodes/catalog/datacatalog/datacatalog.go diff --git a/pkg/controller/nodes/task/catalog/datacatalog/datacatalog_test.go b/pkg/controller/nodes/catalog/datacatalog/datacatalog_test.go similarity index 100% rename from pkg/controller/nodes/task/catalog/datacatalog/datacatalog_test.go rename to pkg/controller/nodes/catalog/datacatalog/datacatalog_test.go diff --git a/pkg/controller/nodes/task/catalog/datacatalog/transformer.go b/pkg/controller/nodes/catalog/datacatalog/transformer.go similarity index 100% rename from pkg/controller/nodes/task/catalog/datacatalog/transformer.go rename to pkg/controller/nodes/catalog/datacatalog/transformer.go diff --git a/pkg/controller/nodes/task/catalog/datacatalog/transformer_test.go b/pkg/controller/nodes/catalog/datacatalog/transformer_test.go similarity index 100% rename from pkg/controller/nodes/task/catalog/datacatalog/transformer_test.go rename to pkg/controller/nodes/catalog/datacatalog/transformer_test.go diff --git a/pkg/controller/nodes/task/catalog/noop_catalog.go b/pkg/controller/nodes/catalog/noop_catalog.go similarity index 100% rename from pkg/controller/nodes/task/catalog/noop_catalog.go rename to pkg/controller/nodes/catalog/noop_catalog.go diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 66d673f5f..69baa20ff 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -561,7 +561,7 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor } else if np == v1alpha1.NodePhaseSkipped { return executors.NodeStatusSuccess, nil } else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - // if cache hit then we immediately process downstream nodes + // if cache hit then immediately process downstream nodes return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } @@ -809,7 +809,7 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - // process downstream nodes + // if cache hit then immediately process downstream nodes return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } From 6471ce886af3e6ad49ab78ea91efc261a05623fb Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 10 Oct 2022 23:46:38 -0500 Subject: [PATCH 18/40] refactored task handler Signed-off-by: Daniel Rammer --- .../task/{pre_post_execution.go => cache.go} | 82 ------------------- pkg/controller/nodes/task/handler.go | 78 ++++++++++++++++++ 2 files changed, 78 insertions(+), 82 deletions(-) rename pkg/controller/nodes/task/{pre_post_execution.go => cache.go} (50%) diff --git a/pkg/controller/nodes/task/pre_post_execution.go b/pkg/controller/nodes/task/cache.go similarity index 50% rename from pkg/controller/nodes/task/pre_post_execution.go rename to pkg/controller/nodes/task/cache.go index 9d0c864e5..f8daa4ad8 100644 --- a/pkg/controller/nodes/task/pre_post_execution.go +++ b/pkg/controller/nodes/task/cache.go @@ -3,13 +3,9 @@ package task import ( "context" - "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" - "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" - "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" errors2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/errors" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" @@ -69,81 +65,3 @@ func (t *Handler) IsCacheable(ctx context.Context, nCtx handler.NodeExecutionCon return taskTemplate.Metadata.Discoverable, taskTemplate.Metadata.Discoverable && taskTemplate.Metadata.CacheSerializable, nil } - -func (t *Handler) ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, - r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, - tr ioutils.SimpleTaskReader) (*io.ExecutionError, error) { - - tk, err := tr.Read(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) - return nil, err - } - - iface := tk.Interface - outputsDeclared := iface != nil && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 - - if r == nil { - if outputsDeclared { - // Whack! plugin did not return any outputs for this task - // Also When an error is observed, cache is automatically disabled - return &io.ExecutionError{ - ExecutionError: &core.ExecutionError{ - Code: "OutputsNotGenerated", - Message: "Output Reader was nil. Plugin/Platform problem.", - }, - IsRecoverable: true, - }, nil - } - return nil, nil - } - // Reader exists, we can check for error, even if this task may not have any outputs declared - y, err := r.IsError(ctx) - if err != nil { - return nil, err - } - if y { - taskErr, err := r.ReadError(ctx) - if err != nil { - return nil, err - } - - if taskErr.ExecutionError == nil { - taskErr.ExecutionError = &core.ExecutionError{Kind: core.ExecutionError_UNKNOWN, Code: "Unknown", Message: "Unknown"} - } - return &taskErr, nil - } - - // Do this if we have outputs declared for the Handler interface! - if !outputsDeclared { - return nil, nil - } - ok, err := r.Exists(ctx) - if err != nil { - logger.Errorf(ctx, "Failed to check if the output file exists. Error: %s", err.Error()) - return nil, err - } - - if !ok { - // Does not exist - return &io.ExecutionError{ - ExecutionError: &core.ExecutionError{ - Code: "OutputsNotFound", - Message: "Outputs not generated by task execution", - }, - IsRecoverable: true, - }, nil - } - - if !r.IsFile(ctx) { - // Read output and write to file - // No need to check for Execution Error here as we have done so above this block. - err = outputCommitter.Put(ctx, r) - if err != nil { - logger.Errorf(ctx, "Failed to commit output to remote location. Error: %v", err) - return nil, err - } - } - - return nil, nil -} diff --git a/pkg/controller/nodes/task/handler.go b/pkg/controller/nodes/task/handler.go index dca47d1d6..9babf602b 100644 --- a/pkg/controller/nodes/task/handler.go +++ b/pkg/controller/nodes/task/handler.go @@ -667,6 +667,84 @@ func (t Handler) Handle(ctx context.Context, nCtx handler.NodeExecutionContext) return pluginTrns.FinalTransition(ctx) } +func (t *Handler) ValidateOutput(ctx context.Context, nodeID v1alpha1.NodeID, i io.InputReader, + r io.OutputReader, outputCommitter io.OutputWriter, executionConfig v1alpha1.ExecutionConfig, + tr ioutils.SimpleTaskReader) (*io.ExecutionError, error) { + + tk, err := tr.Read(ctx) + if err != nil { + logger.Errorf(ctx, "Failed to read TaskTemplate, error :%s", err.Error()) + return nil, err + } + + iface := tk.Interface + outputsDeclared := iface != nil && iface.Outputs != nil && len(iface.Outputs.Variables) > 0 + + if r == nil { + if outputsDeclared { + // Whack! plugin did not return any outputs for this task + // Also When an error is observed, cache is automatically disabled + return &io.ExecutionError{ + ExecutionError: &core.ExecutionError{ + Code: "OutputsNotGenerated", + Message: "Output Reader was nil. Plugin/Platform problem.", + }, + IsRecoverable: true, + }, nil + } + return nil, nil + } + // Reader exists, we can check for error, even if this task may not have any outputs declared + y, err := r.IsError(ctx) + if err != nil { + return nil, err + } + if y { + taskErr, err := r.ReadError(ctx) + if err != nil { + return nil, err + } + + if taskErr.ExecutionError == nil { + taskErr.ExecutionError = &core.ExecutionError{Kind: core.ExecutionError_UNKNOWN, Code: "Unknown", Message: "Unknown"} + } + return &taskErr, nil + } + + // Do this if we have outputs declared for the Handler interface! + if !outputsDeclared { + return nil, nil + } + ok, err := r.Exists(ctx) + if err != nil { + logger.Errorf(ctx, "Failed to check if the output file exists. Error: %s", err.Error()) + return nil, err + } + + if !ok { + // Does not exist + return &io.ExecutionError{ + ExecutionError: &core.ExecutionError{ + Code: "OutputsNotFound", + Message: "Outputs not generated by task execution", + }, + IsRecoverable: true, + }, nil + } + + if !r.IsFile(ctx) { + // Read output and write to file + // No need to check for Execution Error here as we have done so above this block. + err = outputCommitter.Put(ctx, r) + if err != nil { + logger.Errorf(ctx, "Failed to commit output to remote location. Error: %v", err) + return nil, err + } + } + + return nil, nil +} + func (t Handler) Abort(ctx context.Context, nCtx handler.NodeExecutionContext, reason string) error { currentPhase := nCtx.NodeStateReader().GetTaskNodeState().PluginPhase logger.Debugf(ctx, "Abort invoked with phase [%v]", currentPhase) From a0e4619ac9688c178e6c385bd36f300a8df1da06 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 11 Oct 2022 06:28:29 -0500 Subject: [PATCH 19/40] fixed catalog imports on unit testes Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor_test.go | 2 +- pkg/controller/workflow/executor_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/controller/nodes/executor_test.go b/pkg/controller/nodes/executor_test.go index 504ba3ad0..1e6d7b760 100644 --- a/pkg/controller/nodes/executor_test.go +++ b/pkg/controller/nodes/executor_test.go @@ -30,10 +30,10 @@ import ( eventMocks "github.com/flyteorg/flytepropeller/events/mocks" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/mocks" mocks4 "github.com/flyteorg/flytepropeller/pkg/controller/executors/mocks" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/catalog" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" nodeHandlerMocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler/mocks" mocks2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/mocks" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task/catalog" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flytestdlib/promutils" diff --git a/pkg/controller/workflow/executor_test.go b/pkg/controller/workflow/executor_test.go index 46c8be36b..3949976e8 100644 --- a/pkg/controller/workflow/executor_test.go +++ b/pkg/controller/workflow/executor_test.go @@ -32,7 +32,7 @@ import ( eventsErr "github.com/flyteorg/flytepropeller/events/errors" eventMocks "github.com/flyteorg/flytepropeller/events/mocks" mocks2 "github.com/flyteorg/flytepropeller/pkg/controller/executors/mocks" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task/catalog" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/catalog" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task/fakeplugins" wfErrors "github.com/flyteorg/flytepropeller/pkg/controller/workflow/errors" From 439af829cff12df5f7c7e2797700b3cccb64208e Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 11 Oct 2022 07:10:16 -0500 Subject: [PATCH 20/40] started cache unit tests Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 5 +- pkg/controller/nodes/cache_test.go | 75 ++++++++++++++++++++++++++++++ 2 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 pkg/controller/nodes/cache_test.go diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index a71821414..6ef77b1a2 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -2,12 +2,14 @@ package nodes import ( "context" + "strconv" "time" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/encoding" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" @@ -34,7 +36,8 @@ func computeCatalogReservationOwnerID(ctx context.Context, nCtx *nodeExecContext return "", err } - _, ownerID, err := task.ComputeRawOutputPrefix(ctx, task.IDMaxLength, nCtx, currentNodeUniqueID, nCtx.CurrentAttempt()) + ownerID, err := encoding.FixedLengthUniqueIDForParts(task.IDMaxLength, + []string{nCtx.NodeExecutionMetadata().GetOwnerID().Name, currentNodeUniqueID, strconv.Itoa(int(nCtx.CurrentAttempt()))}) if err != nil { return "", err } diff --git a/pkg/controller/nodes/cache_test.go b/pkg/controller/nodes/cache_test.go new file mode 100644 index 000000000..dfd6a9647 --- /dev/null +++ b/pkg/controller/nodes/cache_test.go @@ -0,0 +1,75 @@ +package nodes + +import ( + "context" + "testing" + + "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/mocks" + executorsmocks "github.com/flyteorg/flytepropeller/pkg/controller/executors/mocks" + handlermocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler/mocks" + + "k8s.io/apimachinery/pkg/types" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" +) + +func TestComputeCatalogReservationOwnerID(t *testing.T) { + currentAttempt := uint32(0) + parentUniqueID := "bar" + parentCurrentAttempt := uint32(1) + uniqueID := "foo" + + mockNode := &mocks.ExecutableNode{} + mockNode.OnGetIDMatch(mock.Anything).Return("baz") + + mockNodeStatus := &mocks.ExecutableNodeStatus{} + mockNodeStatus.OnGetAttemptsMatch().Return(currentAttempt) + + mockParentInfo := &executorsmocks.ImmutableParentInfo{} + mockParentInfo.OnCurrentAttemptMatch().Return(parentCurrentAttempt) + mockParentInfo.OnGetUniqueIDMatch().Return(uniqueID) + + mockExecutionContext := &executorsmocks.ExecutionContext{} + mockExecutionContext.OnGetParentInfoMatch(mock.Anything).Return(mockParentInfo) + + mockNodeExecutionMetadata := &handlermocks.NodeExecutionMetadata{} + mockNodeExecutionMetadata.OnGetOwnerID().Return( + types.NamespacedName{ + Name: parentUniqueID, + }, + ) + + nCtx := &nodeExecContext{ + ic: mockExecutionContext, + md: mockNodeExecutionMetadata, + node: mockNode, + nodeStatus: mockNodeStatus, + } + + ownerID, err := computeCatalogReservationOwnerID(context.TODO(), nCtx) + assert.NoError(t, err) + assert.Equal(t, "bar-foo-1-baz-0", ownerID) +} + +func TestUpdatePhaseCacheInfo(t *testing.T) { + //func updatePhaseCacheInfo(phaseInfo handler.PhaseInfo, cacheStatus *catalog.Status, reservationStatus *core.CatalogReservation_Status) handler.PhaseInfo { +} + +func TestCheckCatalogCache(t *testing.T) { + //func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { +} + +func TestGetOrExtendCatalogReservation(t *testing.T) { + //func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx *nodeExecContext, + // cacheHandler handler.CacheableNode, heartbeatInterval time.Duration) (catalog.ReservationEntry, error) { +} + +func TestReleaseCatalogReservation(t *testing.T) { + //func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *nodeExecContext, + // cacheHandler handler.CacheableNode) (catalog.ReservationEntry, error) { +} + +func TestWriteCatalogCache(t *testing.T) { + //func (n *nodeExecutor) WriteCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Status, error) { +} From 02e314c38b73aa76b600b0dd0055b0b317291ce8 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 11 Oct 2022 11:22:43 -0500 Subject: [PATCH 21/40] added CheckCatalogCache unit tests Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache_test.go | 272 +++++++++++++++++++++++++++-- 1 file changed, 259 insertions(+), 13 deletions(-) diff --git a/pkg/controller/nodes/cache_test.go b/pkg/controller/nodes/cache_test.go index dfd6a9647..6d528a7de 100644 --- a/pkg/controller/nodes/cache_test.go +++ b/pkg/controller/nodes/cache_test.go @@ -2,29 +2,54 @@ package nodes import ( "context" + "errors" + "fmt" "testing" + "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + + eventsmocks "github.com/flyteorg/flytepropeller/events/mocks" + "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/mocks" + "github.com/flyteorg/flytepropeller/pkg/controller/config" executorsmocks "github.com/flyteorg/flytepropeller/pkg/controller/executors/mocks" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" handlermocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler/mocks" + recoverymocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/recovery/mocks" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" + + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + catalogmocks "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog/mocks" + "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" + + "github.com/flyteorg/flytestdlib/promutils" + "github.com/flyteorg/flytestdlib/storage" "k8s.io/apimachinery/pkg/types" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) -func TestComputeCatalogReservationOwnerID(t *testing.T) { - currentAttempt := uint32(0) - parentUniqueID := "bar" - parentCurrentAttempt := uint32(1) - uniqueID := "foo" +var ( + currentAttempt = uint32(0) + nodeID = "baz" + nodeOutputDir = storage.DataReference("output_directory") + parentUniqueID = "bar" + parentCurrentAttempt = uint32(1) + uniqueID = "foo" +) +func setupCacheableNodeExecutionContext(dataStore *storage.DataStore) *nodeExecContext { mockNode := &mocks.ExecutableNode{} - mockNode.OnGetIDMatch(mock.Anything).Return("baz") + mockNode.OnGetIDMatch(mock.Anything).Return(nodeID) mockNodeStatus := &mocks.ExecutableNodeStatus{} mockNodeStatus.OnGetAttemptsMatch().Return(currentAttempt) + mockNodeStatus.OnGetOutputDir().Return(nodeOutputDir) mockParentInfo := &executorsmocks.ImmutableParentInfo{} mockParentInfo.OnCurrentAttemptMatch().Return(parentCurrentAttempt) @@ -40,24 +65,245 @@ func TestComputeCatalogReservationOwnerID(t *testing.T) { }, ) - nCtx := &nodeExecContext{ - ic: mockExecutionContext, - md: mockNodeExecutionMetadata, - node: mockNode, + return &nodeExecContext{ + ic: mockExecutionContext, + md: mockNodeExecutionMetadata, + node: mockNode, nodeStatus: mockNodeStatus, + store: dataStore, } +} + +func setupCacheableNodeExecutor(t *testing.T, catalogClient catalog.Client, dataStore *storage.DataStore, testScope promutils.Scope) *nodeExecutor { + ctx := context.TODO() + + adminClient := launchplan.NewFailFastLaunchPlanExecutor() + enqueueWorkflow := func(workflowID v1alpha1.WorkflowID) {} + eventConfig := &config.EventConfig{ + RawOutputPolicy: config.RawOutputPolicyReference, + } + fakeKubeClient := executorsmocks.NewFakeKubeClient() + maxDatasetSize := int64(10) + mockEventSink := eventsmocks.NewMockEventSink() + nodeConfig := config.GetConfig().NodeConfig + rawOutputPrefix := storage.DataReference("s3://bucket/") + recoveryClient := &recoverymocks.Client{} + testClusterID := "cluster1" + + nodeExecutorInterface, err := NewExecutor(ctx, nodeConfig, dataStore, enqueueWorkflow, mockEventSink, + adminClient, adminClient, maxDatasetSize, rawOutputPrefix, fakeKubeClient, catalogClient, + recoveryClient, eventConfig, testClusterID, testScope.NewSubScope("node_executor")) + assert.NoError(t, err) + + nodeExecutor, ok := nodeExecutorInterface.(*nodeExecutor) + assert.True(t, ok) + + return nodeExecutor +} + +func TestComputeCatalogReservationOwnerID(t *testing.T) { + nCtx := setupCacheableNodeExecutionContext(nil) ownerID, err := computeCatalogReservationOwnerID(context.TODO(), nCtx) assert.NoError(t, err) - assert.Equal(t, "bar-foo-1-baz-0", ownerID) + assert.Equal(t, fmt.Sprintf("%s-%s-%d-%s-%d", parentUniqueID, uniqueID, parentCurrentAttempt, nodeID, currentAttempt), ownerID) } func TestUpdatePhaseCacheInfo(t *testing.T) { - //func updatePhaseCacheInfo(phaseInfo handler.PhaseInfo, cacheStatus *catalog.Status, reservationStatus *core.CatalogReservation_Status) handler.PhaseInfo { + cacheStatus := catalog.NewStatus(core.CatalogCacheStatus_CACHE_MISS, nil) + reservationStatus := core.CatalogReservation_RESERVATION_EXISTS + + tests := []struct { + name string + cacheStatus *catalog.Status + reservationStatus *core.CatalogReservation_Status + }{ + {"BothEmpty", nil, nil}, + {"CacheStatusOnly", &cacheStatus, nil}, + {"ReservationStatusOnly", nil, &reservationStatus}, + {"BothPopulated", &cacheStatus, &reservationStatus}, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + phaseInfo := handler.PhaseInfoUndefined + phaseInfo = updatePhaseCacheInfo(phaseInfo, test.cacheStatus, test.reservationStatus) + + // do not create ExecutionInfo object if neither cacheStatus or reservationStatus exists + if test.cacheStatus == nil && test.reservationStatus == nil { + assert.Nil(t, phaseInfo.GetInfo()) + } + + // ensure cache and reservation status' are being set correctly + if test.cacheStatus != nil { + assert.Equal(t, cacheStatus.GetCacheStatus(), phaseInfo.GetInfo().TaskNodeInfo.TaskNodeMetadata.CacheStatus) + } + + if test.reservationStatus != nil { + assert.Equal(t, reservationStatus, phaseInfo.GetInfo().TaskNodeInfo.TaskNodeMetadata.ReservationStatus) + } + }) + } } func TestCheckCatalogCache(t *testing.T) { - //func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Entry, error) { + t.Run("CacheMiss", func(t *testing.T) { + testScope := promutils.NewTestScope() + + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) + + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.Entry{}, status.Error(codes.NotFound, "")) + + dataStore, err := storage.NewDataStore( + &storage.Config{ + Type: storage.TypeMemory, + }, + testScope.NewSubScope("data_store"), + ) + assert.NoError(t, err) + + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nCtx := setupCacheableNodeExecutionContext(dataStore) + + resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) + assert.NoError(t, err) + + assert.Equal(t, core.CatalogCacheStatus_CACHE_MISS, resultCacheEntry.GetStatus().GetCacheStatus()) + }) + + t.Run("CacheHitWithOutputs", func(t *testing.T) { + testScope := promutils.NewTestScope() + cacheEntry := catalog.NewCatalogEntry( + ioutils.NewInMemoryOutputReader(&core.LiteralMap{}, nil, nil), + catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil), + ) + catalogKey := catalog.Key{ + TypedInterface: core.TypedInterface{ + Outputs: &core.VariableMap{ + Variables: map[string]*core.Variable{ + "foo": nil, + }, + }, + }, + } + + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalogKey, nil) + + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(cacheEntry, nil) + + dataStore, err := storage.NewDataStore( + &storage.Config{ + Type: storage.TypeMemory, + }, + testScope.NewSubScope("data_store"), + ) + assert.NoError(t, err) + + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nCtx := setupCacheableNodeExecutionContext(dataStore) + + resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) + assert.NoError(t, err) + + assert.Equal(t, core.CatalogCacheStatus_CACHE_HIT, resultCacheEntry.GetStatus().GetCacheStatus()) + + // assert the outputs file exists + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + metadata, err := nCtx.DataStore().Head(context.TODO(), outputFile) + assert.NoError(t, err) + assert.Equal(t, true, metadata.Exists()) + }) + + t.Run("CacheHitWithoutOutputs", func(t *testing.T) { + testScope := promutils.NewTestScope() + cacheEntry := catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil)) + + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) + + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(cacheEntry, nil) + + dataStore, err := storage.NewDataStore( + &storage.Config{ + Type: storage.TypeMemory, + }, + testScope.NewSubScope("data_store"), + ) + assert.NoError(t, err) + + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nCtx := setupCacheableNodeExecutionContext(dataStore) + + resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) + assert.NoError(t, err) + + assert.Equal(t, core.CatalogCacheStatus_CACHE_HIT, resultCacheEntry.GetStatus().GetCacheStatus()) + + // assert the outputs file does not exist + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + metadata, err := nCtx.DataStore().Head(context.TODO(), outputFile) + assert.NoError(t, err) + assert.Equal(t, false, metadata.Exists()) + }) + + t.Run("CacheLookupError", func(t *testing.T) { + testScope := promutils.NewTestScope() + + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) + + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.Entry{}, errors.New("foo")) + + dataStore, err := storage.NewDataStore( + &storage.Config{ + Type: storage.TypeMemory, + }, + testScope.NewSubScope("data_store"), + ) + assert.NoError(t, err) + + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nCtx := setupCacheableNodeExecutionContext(dataStore) + + _, err = nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) + assert.Error(t, err) + }) + + t.Run("OutputsAlreadyExist", func(t *testing.T) { + testScope := promutils.NewTestScope() + + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) + + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.Entry{}, nil) + + dataStore, err := storage.NewDataStore( + &storage.Config{ + Type: storage.TypeMemory, + }, + testScope.NewSubScope("data_store"), + ) + assert.NoError(t, err) + + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nCtx := setupCacheableNodeExecutionContext(dataStore) + + // write mock data to outputs + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + err = nCtx.DataStore().WriteProtobuf(context.TODO(), outputFile, storage.Options{}, &core.LiteralMap{}) + assert.NoError(t, err) + + resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) + assert.NoError(t, err) + + assert.Equal(t, core.CatalogCacheStatus_CACHE_HIT, resultCacheEntry.GetStatus().GetCacheStatus()) + }) } func TestGetOrExtendCatalogReservation(t *testing.T) { From 4b8c241ed65ac1a83d2a9f8fc17b670fb63931c9 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 11 Oct 2022 13:40:09 -0500 Subject: [PATCH 22/40] unit tests for node cache file Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 6 +- pkg/controller/nodes/cache_test.go | 411 ++++++++++++++++++----------- 2 files changed, 267 insertions(+), 150 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 6ef77b1a2..6334c3a05 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -30,7 +30,7 @@ import ( // computeCatalogReservationOwnerID constructs a unique identifier which includes the nodes // parent information, node ID, and retry attempt number. This is used to uniquely identify a task // when the cache reservation API to serialize cached executions. -func computeCatalogReservationOwnerID(ctx context.Context, nCtx *nodeExecContext) (string, error) { +func computeCatalogReservationOwnerID(nCtx *nodeExecContext) (string, error) { currentNodeUniqueID, err := common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) if err != nil { return "", err @@ -157,7 +157,7 @@ func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx * errors.Wrapf(err, "failed to initialize the catalogKey") } - ownerID, err := computeCatalogReservationOwnerID(ctx, nCtx) + ownerID, err := computeCatalogReservationOwnerID(nCtx) if err != nil { return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), errors.Wrapf(err, "failed to initialize the cache reservation ownerID") @@ -194,7 +194,7 @@ func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *node errors.Wrapf(err, "failed to initialize the catalogKey") } - ownerID, err := computeCatalogReservationOwnerID(ctx, nCtx) + ownerID, err := computeCatalogReservationOwnerID(nCtx) if err != nil { return catalog.NewReservationEntryStatus(core.CatalogReservation_RESERVATION_DISABLED), errors.Wrapf(err, "failed to initialize the cache reservation ownerID") diff --git a/pkg/controller/nodes/cache_test.go b/pkg/controller/nodes/cache_test.go index 6d528a7de..166fccfd8 100644 --- a/pkg/controller/nodes/cache_test.go +++ b/pkg/controller/nodes/cache_test.go @@ -5,8 +5,10 @@ import ( "errors" "fmt" "testing" + "time" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/datacatalog" eventsmocks "github.com/flyteorg/flytepropeller/events/mocks" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" @@ -35,15 +37,25 @@ import ( ) var ( - currentAttempt = uint32(0) - nodeID = "baz" - nodeOutputDir = storage.DataReference("output_directory") - parentUniqueID = "bar" + currentAttempt = uint32(0) + nodeID = "baz" + nodeOutputDir = storage.DataReference("output_directory") + parentUniqueID = "bar" parentCurrentAttempt = uint32(1) - uniqueID = "foo" + uniqueID = "foo" ) -func setupCacheableNodeExecutionContext(dataStore *storage.DataStore) *nodeExecContext { +type mockTaskReader struct { + taskTemplate *core.TaskTemplate +} + +func (t mockTaskReader) Read(ctx context.Context) (*core.TaskTemplate, error) { + return t.taskTemplate, nil +} +func (t mockTaskReader) GetTaskType() v1alpha1.TaskType { return "" } +func (t mockTaskReader) GetTaskID() *core.Identifier { return nil } + +func setupCacheableNodeExecutionContext(dataStore *storage.DataStore, taskTemplate *core.TaskTemplate) *nodeExecContext { mockNode := &mocks.ExecutableNode{} mockNode.OnGetIDMatch(mock.Anything).Return(nodeID) @@ -64,6 +76,18 @@ func setupCacheableNodeExecutionContext(dataStore *storage.DataStore) *nodeExecC Name: parentUniqueID, }, ) + mockNodeExecutionMetadata.OnGetNodeExecutionIDMatch().Return( + &core.NodeExecutionIdentifier{ + NodeId: nodeID, + }, + ) + + var taskReader handler.TaskReader + if taskTemplate != nil { + taskReader = mockTaskReader{ + taskTemplate: taskTemplate, + } + } return &nodeExecContext{ ic: mockExecutionContext, @@ -71,6 +95,7 @@ func setupCacheableNodeExecutionContext(dataStore *storage.DataStore) *nodeExecC node: mockNode, nodeStatus: mockNodeStatus, store: dataStore, + tr: taskReader, } } @@ -102,9 +127,9 @@ func setupCacheableNodeExecutor(t *testing.T, catalogClient catalog.Client, data } func TestComputeCatalogReservationOwnerID(t *testing.T) { - nCtx := setupCacheableNodeExecutionContext(nil) + nCtx := setupCacheableNodeExecutionContext(nil, nil) - ownerID, err := computeCatalogReservationOwnerID(context.TODO(), nCtx) + ownerID, err := computeCatalogReservationOwnerID(nCtx) assert.NoError(t, err) assert.Equal(t, fmt.Sprintf("%s-%s-%d-%s-%d", parentUniqueID, uniqueID, parentCurrentAttempt, nodeID, currentAttempt), ownerID) } @@ -146,176 +171,268 @@ func TestUpdatePhaseCacheInfo(t *testing.T) { } func TestCheckCatalogCache(t *testing.T) { - t.Run("CacheMiss", func(t *testing.T) { - testScope := promutils.NewTestScope() - - cacheableHandler := &handlermocks.CacheableNode{} - cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) - - catalogClient := &catalogmocks.Client{} - catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.Entry{}, status.Error(codes.NotFound, "")) - - dataStore, err := storage.NewDataStore( - &storage.Config{ - Type: storage.TypeMemory, - }, - testScope.NewSubScope("data_store"), - ) - assert.NoError(t, err) - - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) - nCtx := setupCacheableNodeExecutionContext(dataStore) - - resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) - assert.NoError(t, err) - - assert.Equal(t, core.CatalogCacheStatus_CACHE_MISS, resultCacheEntry.GetStatus().GetCacheStatus()) - }) - - t.Run("CacheHitWithOutputs", func(t *testing.T) { - testScope := promutils.NewTestScope() - cacheEntry := catalog.NewCatalogEntry( - ioutils.NewInMemoryOutputReader(&core.LiteralMap{}, nil, nil), - catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil), - ) - catalogKey := catalog.Key{ - TypedInterface: core.TypedInterface{ - Outputs: &core.VariableMap{ - Variables: map[string]*core.Variable{ - "foo": nil, + tests := []struct { + name string + cacheEntry catalog.Entry + cacheError error + catalogKey catalog.Key + expectedCacheStatus core.CatalogCacheStatus + preWriteOutputFile bool + assertOutputFile bool + outputFileExists bool + }{ + { + "CacheMiss", + catalog.Entry{}, + status.Error(codes.NotFound, ""), + catalog.Key{}, + core.CatalogCacheStatus_CACHE_MISS, + false, + false, + false, + }, + { + "CacheHitWithOutputs", + catalog.NewCatalogEntry( + ioutils.NewInMemoryOutputReader(&core.LiteralMap{}, nil, nil), + catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil), + ), + nil, + catalog.Key{ + TypedInterface: core.TypedInterface{ + Outputs: &core.VariableMap{ + Variables: map[string]*core.Variable{ + "foo": nil, + }, }, }, }, - } - - cacheableHandler := &handlermocks.CacheableNode{} - cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalogKey, nil) - - catalogClient := &catalogmocks.Client{} - catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(cacheEntry, nil) - - dataStore, err := storage.NewDataStore( - &storage.Config{ - Type: storage.TypeMemory, - }, - testScope.NewSubScope("data_store"), - ) - assert.NoError(t, err) + core.CatalogCacheStatus_CACHE_HIT, + false, + true, + true, + }, + { + "CacheHitWithoutOutputs", + catalog.NewCatalogEntry( + nil, + catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil), + ), + nil, + catalog.Key{}, + core.CatalogCacheStatus_CACHE_HIT, + false, + true, + false, + }, + { + "OutputsAlreadyExist", + catalog.Entry{}, + nil, + catalog.Key{}, + core.CatalogCacheStatus_CACHE_HIT, + true, + true, + true, + }, + } - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) - nCtx := setupCacheableNodeExecutionContext(dataStore) + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + testScope := promutils.NewTestScope() - resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) - assert.NoError(t, err) + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(test.catalogKey, nil) - assert.Equal(t, core.CatalogCacheStatus_CACHE_HIT, resultCacheEntry.GetStatus().GetCacheStatus()) + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(test.cacheEntry, test.cacheError) - // assert the outputs file exists - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) - metadata, err := nCtx.DataStore().Head(context.TODO(), outputFile) - assert.NoError(t, err) - assert.Equal(t, true, metadata.Exists()) - }) + dataStore, err := storage.NewDataStore( + &storage.Config{ + Type: storage.TypeMemory, + }, + testScope.NewSubScope("data_store"), + ) + assert.NoError(t, err) + + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nCtx := setupCacheableNodeExecutionContext(dataStore, nil) + + if test.preWriteOutputFile { + // write mock data to outputs + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + err = nCtx.DataStore().WriteProtobuf(context.TODO(), outputFile, storage.Options{}, &core.LiteralMap{}) + assert.NoError(t, err) + } - t.Run("CacheHitWithoutOutputs", func(t *testing.T) { - testScope := promutils.NewTestScope() - cacheEntry := catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil)) + // execute catalog cache check + cacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) + assert.NoError(t, err) - cacheableHandler := &handlermocks.CacheableNode{} - cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) + // validate the result cache entry status + assert.Equal(t, test.expectedCacheStatus, cacheEntry.GetStatus().GetCacheStatus()) - catalogClient := &catalogmocks.Client{} - catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(cacheEntry, nil) + if test.assertOutputFile { + // assert the outputs file exists + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) + metadata, err := nCtx.DataStore().Head(context.TODO(), outputFile) + assert.NoError(t, err) + assert.Equal(t, test.outputFileExists, metadata.Exists()) + } + }) + } +} - dataStore, err := storage.NewDataStore( - &storage.Config{ - Type: storage.TypeMemory, - }, - testScope.NewSubScope("data_store"), - ) - assert.NoError(t, err) +func TestGetOrExtendCatalogReservation(t *testing.T) { + tests := []struct { + name string + reservationOwnerID string + expectedReservationStatus core.CatalogReservation_Status + }{ + { + "Acquired", + "bar-foo-1-baz-0", + core.CatalogReservation_RESERVATION_ACQUIRED, + }, + { + "Exists", + "some-other-owner", + core.CatalogReservation_RESERVATION_EXISTS, + }, + } - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) - nCtx := setupCacheableNodeExecutionContext(dataStore) + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + testScope := promutils.NewTestScope() - resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) - assert.NoError(t, err) + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) - assert.Equal(t, core.CatalogCacheStatus_CACHE_HIT, resultCacheEntry.GetStatus().GetCacheStatus()) + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetOrExtendReservationMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return( + &datacatalog.Reservation{ + OwnerId: test.reservationOwnerID, + }, + nil, + ) - // assert the outputs file does not exist - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) - metadata, err := nCtx.DataStore().Head(context.TODO(), outputFile) - assert.NoError(t, err) - assert.Equal(t, false, metadata.Exists()) - }) + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, nil, testScope) + nCtx := setupCacheableNodeExecutionContext(nil, &core.TaskTemplate{}) - t.Run("CacheLookupError", func(t *testing.T) { - testScope := promutils.NewTestScope() + // execute catalog cache check + reservationEntry, err := nodeExecutor.GetOrExtendCatalogReservation(context.TODO(), nCtx, cacheableHandler, time.Second*30) + assert.NoError(t, err) - cacheableHandler := &handlermocks.CacheableNode{} - cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) + // validate the result cache entry status + assert.Equal(t, test.expectedReservationStatus, reservationEntry.GetStatus()) + }) + } +} - catalogClient := &catalogmocks.Client{} - catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.Entry{}, errors.New("foo")) +func TestReleaseCatalogReservation(t *testing.T) { + tests := []struct { + name string + releaseError error + expectedReservationStatus core.CatalogReservation_Status + }{ + { + "Success", + nil, + core.CatalogReservation_RESERVATION_RELEASED, + }, + { + "Failure", + errors.New("failed to release"), + core.CatalogReservation_RESERVATION_FAILURE, + }, + } - dataStore, err := storage.NewDataStore( - &storage.Config{ - Type: storage.TypeMemory, - }, - testScope.NewSubScope("data_store"), - ) - assert.NoError(t, err) + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + testScope := promutils.NewTestScope() - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) - nCtx := setupCacheableNodeExecutionContext(dataStore) + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) - _, err = nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) - assert.Error(t, err) - }) + catalogClient := &catalogmocks.Client{} + catalogClient.OnReleaseReservationMatch(mock.Anything, mock.Anything, mock.Anything).Return(test.releaseError) - t.Run("OutputsAlreadyExist", func(t *testing.T) { - testScope := promutils.NewTestScope() + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, nil, testScope) + nCtx := setupCacheableNodeExecutionContext(nil, &core.TaskTemplate{}) - cacheableHandler := &handlermocks.CacheableNode{} - cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) + // execute catalog cache check + reservationEntry, err := nodeExecutor.ReleaseCatalogReservation(context.TODO(), nCtx, cacheableHandler) + if test.releaseError == nil { + assert.NoError(t, err) + } else { + assert.Error(t, err) + } - catalogClient := &catalogmocks.Client{} - catalogClient.OnGetMatch(mock.Anything, mock.Anything).Return(catalog.Entry{}, nil) + // validate the result cache entry status + assert.Equal(t, test.expectedReservationStatus, reservationEntry.GetStatus()) + }) + } +} - dataStore, err := storage.NewDataStore( - &storage.Config{ - Type: storage.TypeMemory, +func TestWriteCatalogCache(t *testing.T) { + tests := []struct { + name string + cacheStatus catalog.Status + cacheError error + catalogKey catalog.Key + expectedCacheStatus core.CatalogCacheStatus + }{ + { + "NoOutputs", + catalog.NewStatus(core.CatalogCacheStatus_CACHE_DISABLED, nil), + nil, + catalog.Key{}, + core.CatalogCacheStatus_CACHE_DISABLED, + }, + { + "OutputsExist", + catalog.NewStatus(core.CatalogCacheStatus_CACHE_POPULATED, nil), + nil, + catalog.Key{ + TypedInterface: core.TypedInterface{ + Outputs: &core.VariableMap{ + Variables: map[string]*core.Variable{ + "foo": nil, + }, + }, + }, }, - testScope.NewSubScope("data_store"), - ) - assert.NoError(t, err) + core.CatalogCacheStatus_CACHE_POPULATED, + }, + } - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) - nCtx := setupCacheableNodeExecutionContext(dataStore) + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + testScope := promutils.NewTestScope() - // write mock data to outputs - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) - err = nCtx.DataStore().WriteProtobuf(context.TODO(), outputFile, storage.Options{}, &core.LiteralMap{}) - assert.NoError(t, err) + cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(test.catalogKey, nil) - resultCacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) - assert.NoError(t, err) + catalogClient := &catalogmocks.Client{} + catalogClient.OnPutMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(test.cacheStatus, nil) - assert.Equal(t, core.CatalogCacheStatus_CACHE_HIT, resultCacheEntry.GetStatus().GetCacheStatus()) - }) -} + dataStore, err := storage.NewDataStore( + &storage.Config{ + Type: storage.TypeMemory, + }, + testScope.NewSubScope("data_store"), + ) + assert.NoError(t, err) -func TestGetOrExtendCatalogReservation(t *testing.T) { - //func (n *nodeExecutor) GetOrExtendCatalogReservation(ctx context.Context, nCtx *nodeExecContext, - // cacheHandler handler.CacheableNode, heartbeatInterval time.Duration) (catalog.ReservationEntry, error) { -} + nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nCtx := setupCacheableNodeExecutionContext(dataStore, &core.TaskTemplate{}) -func TestReleaseCatalogReservation(t *testing.T) { - //func (n *nodeExecutor) ReleaseCatalogReservation(ctx context.Context, nCtx *nodeExecContext, - // cacheHandler handler.CacheableNode) (catalog.ReservationEntry, error) { -} + // execute catalog cache check + cacheStatus, err := nodeExecutor.WriteCatalogCache(context.TODO(), nCtx, cacheableHandler) + assert.NoError(t, err) -func TestWriteCatalogCache(t *testing.T) { - //func (n *nodeExecutor) WriteCatalogCache(ctx context.Context, nCtx *nodeExecContext, cacheHandler handler.CacheableNode) (catalog.Status, error) { + // validate the result cache entry status + assert.Equal(t, test.expectedCacheStatus, cacheStatus.GetCacheStatus()) + }) + } } From 757d3b3b00459f332a201a91a694f279cae16f19 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 11 Oct 2022 21:33:21 -0500 Subject: [PATCH 23/40] added node executor cache unit tests Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 7 + pkg/controller/nodes/executor_test.go | 271 ++++++++++++++++++++++++-- 2 files changed, 258 insertions(+), 20 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 69baa20ff..29abad110 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -534,6 +534,11 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "failed to move from queued") } + if np == v1alpha1.NodePhaseSucceeding && !h.FinalizeRequired() { + logger.Infof(ctx, "Finalize not required, moving node to Succeeded") + np = v1alpha1.NodePhaseSucceeded + } + if np != nodeStatus.GetPhase() { // assert np == Queued! logger.Infof(ctx, "Change in node state detected from [%s] -> [%s]", nodeStatus.GetPhase().String(), np.String()) @@ -562,6 +567,7 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusSuccess, nil } else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { // if cache hit then immediately process downstream nodes + nodeStatus.ResetDirty() return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } @@ -810,6 +816,7 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { // if cache hit then immediately process downstream nodes + nodeStatus.ResetDirty() return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } diff --git a/pkg/controller/nodes/executor_test.go b/pkg/controller/nodes/executor_test.go index 1e6d7b760..e6dc89ad9 100644 --- a/pkg/controller/nodes/executor_test.go +++ b/pkg/controller/nodes/executor_test.go @@ -8,43 +8,45 @@ import ( "testing" "time" - "github.com/flyteorg/flytestdlib/contextutils" - - "github.com/golang/protobuf/proto" - - mocks3 "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io/mocks" - storageMocks "github.com/flyteorg/flytestdlib/storage/mocks" - "github.com/flyteorg/flyteidl/clients/go/coreutils" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/admin" - + "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" - "github.com/flyteorg/flytestdlib/promutils/labeled" - "github.com/flyteorg/flytestdlib/storage" - "github.com/prometheus/client_golang/prometheus" - "github.com/stretchr/testify/mock" - v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + mocks3 "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io/mocks" + pluginscatalog "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + catalogmocks "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog/mocks" "github.com/flyteorg/flytepropeller/events" eventsErr "github.com/flyteorg/flytepropeller/events/errors" eventMocks "github.com/flyteorg/flytepropeller/events/mocks" + "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/mocks" + "github.com/flyteorg/flytepropeller/pkg/controller/config" + "github.com/flyteorg/flytepropeller/pkg/controller/executors" mocks4 "github.com/flyteorg/flytepropeller/pkg/controller/executors/mocks" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/catalog" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" nodeHandlerMocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler/mocks" mocks2 "github.com/flyteorg/flytepropeller/pkg/controller/nodes/mocks" + recoveryMocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/recovery/mocks" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" + flyteassert "github.com/flyteorg/flytepropeller/pkg/utils/assert" - "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flytestdlib/contextutils" "github.com/flyteorg/flytestdlib/promutils" + "github.com/flyteorg/flytestdlib/promutils/labeled" + "github.com/flyteorg/flytestdlib/storage" + storageMocks "github.com/flyteorg/flytestdlib/storage/mocks" + + "github.com/golang/protobuf/proto" + + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" - "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" - "github.com/flyteorg/flytepropeller/pkg/controller/config" - "github.com/flyteorg/flytepropeller/pkg/controller/executors" - recoveryMocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/recovery/mocks" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" - flyteassert "github.com/flyteorg/flytepropeller/pkg/utils/assert" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) var fakeKubeClient = mocks4.NewFakeKubeClient() @@ -2475,3 +2477,232 @@ func (e existsMetadata) Exists() bool { func (e existsMetadata) Size() int64 { return int64(1) } + +func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { + currentNodeID := "node-0" + downstreamNodeID := "node-1" + taskID := taskID + + createMockWorkflow := func(currentNodePhase, downstreamNodePhase v1alpha1.NodePhase, dataStore *storage.DataStore) *v1alpha1.FlyteWorkflow { + currentNodeSpec := &v1alpha1.NodeSpec{ + ID: currentNodeID, + TaskRef: &taskID, + Kind: v1alpha1.NodeKindTask, + } + + /*var err *v1alpha1.ExecutionError + if p == v1alpha1.NodePhaseFailing || p == v1alpha1.NodePhaseFailed { + err = &v1alpha1.ExecutionError{ExecutionError: &core.ExecutionError{Code: "test", Message: "test"}} + }*/ + currentNodeStatus := &v1alpha1.NodeStatus{ + Phase: currentNodePhase, + //LastAttemptStartedAt: &v1.Time{}, + //Error: err, + } + + downstreamNodeSpec := &v1alpha1.NodeSpec{ + ID: downstreamNodeID, + TaskRef: &taskID, + Kind: v1alpha1.NodeKindTask, + } + + downstreamNodeStatus := &v1alpha1.NodeStatus{ + Phase: downstreamNodePhase, + } + + return &v1alpha1.FlyteWorkflow{ + Tasks: map[v1alpha1.TaskID]*v1alpha1.TaskSpec{ + taskID: { + TaskTemplate: &core.TaskTemplate{}, + }, + }, + Status: v1alpha1.WorkflowStatus{ + NodeStatus: map[v1alpha1.NodeID]*v1alpha1.NodeStatus{ + currentNodeID: currentNodeStatus, + downstreamNodeID: downstreamNodeStatus, + }, + DataDir: "data", + }, + WorkflowSpec: &v1alpha1.WorkflowSpec{ + ID: "wf", + Nodes: map[v1alpha1.NodeID]*v1alpha1.NodeSpec{ + currentNodeID: currentNodeSpec, + downstreamNodeID: downstreamNodeSpec, + }, + Connections: v1alpha1.Connections{ + Upstream: map[v1alpha1.NodeID][]v1alpha1.NodeID{ + downstreamNodeID: {currentNodeID}, + }, + Downstream: map[v1alpha1.NodeID][]v1alpha1.NodeID{ + currentNodeID: {downstreamNodeID}, + }, + }, + }, + DataReferenceConstructor: dataStore, + RawOutputDataConfig: v1alpha1.RawOutputDataConfig{ + RawOutputDataConfig: &admin.RawOutputDataConfig{OutputLocationPrefix: ""}, + }, + } + } + + setupNodeExecutor := func(t *testing.T, catalogClient pluginscatalog.Client, dataStore *storage.DataStore, mockHandler handler.CacheableNode, testScope promutils.Scope) *nodeExecutor { + ctx := context.TODO() + + // create mocks + adminClient := launchplan.NewFailFastLaunchPlanExecutor() + enqueueWorkflow := func(workflowID v1alpha1.WorkflowID) {} + eventConfig := &config.EventConfig{ + RawOutputPolicy: config.RawOutputPolicyReference, + } + fakeKubeClient := mocks4.NewFakeKubeClient() + maxDatasetSize := int64(10) + mockEventSink := eventMocks.NewMockEventSink() + nodeConfig := config.GetConfig().NodeConfig + rawOutputPrefix := storage.DataReference("s3://bucket/") + recoveryClient := &recoveryMocks.Client{} + testClusterID := "cluster1" + + // initialize node executor + nodeExecutorInterface, err := NewExecutor(ctx, nodeConfig, dataStore, enqueueWorkflow, mockEventSink, + adminClient, adminClient, maxDatasetSize, rawOutputPrefix, fakeKubeClient, catalogClient, + recoveryClient, eventConfig, testClusterID, testScope) + assert.NoError(t, err) + + nodeExecutor, ok := nodeExecutorInterface.(*nodeExecutor) + assert.True(t, ok) + + // setup node handler + mockHandlerFactory := &mocks2.HandlerFactory{} + mockHandlerFactory.OnGetHandler(v1alpha1.NodeKindTask).Return(mockHandler, nil) + nodeExecutor.nodeHandlerFactory = mockHandlerFactory + + return nodeExecutor + } + + tests := []struct { + name string + cacheable bool + cacheStatus core.CatalogCacheStatus + cacheSerializable bool + cacheSerializeStatus core.CatalogReservation_Status + currentNodePhase v1alpha1.NodePhase + nextNodePhase v1alpha1.NodePhase + currentDownstreamNodePhase v1alpha1.NodePhase + nextDownstreamNodePhase v1alpha1.NodePhase + }{ + { + "NotYetStarted->CacheMiss->Queued", + true, + core.CatalogCacheStatus_CACHE_MISS, + false, + core.CatalogReservation_RESERVATION_DISABLED, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseQueued, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseNotYetStarted, + }, + { + "NotYetStarted->CacheHit->Success", + true, + core.CatalogCacheStatus_CACHE_HIT, + false, + core.CatalogReservation_RESERVATION_DISABLED, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseSucceeded, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseQueued, + }, + { + "Queued->CacheHit->Success", + true, + core.CatalogCacheStatus_CACHE_HIT, + true, + core.CatalogReservation_RESERVATION_EXISTS, + v1alpha1.NodePhaseQueued, + v1alpha1.NodePhaseSucceeded, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseQueued, + }, + { + "Queued->CacheMiss->Queued", + true, + core.CatalogCacheStatus_CACHE_MISS, + true, + core.CatalogReservation_RESERVATION_EXISTS, + v1alpha1.NodePhaseQueued, + v1alpha1.NodePhaseQueued, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseNotYetStarted, + }, + { + "Queued->ReservationAcquired->Running", + true, + core.CatalogCacheStatus_CACHE_MISS, + true, + core.CatalogReservation_RESERVATION_ACQUIRED, + v1alpha1.NodePhaseQueued, + v1alpha1.NodePhaseRunning, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseNotYetStarted, + }, + { + "Running->ReservationExists->Running", + true, + core.CatalogCacheStatus_CACHE_MISS, + true, + core.CatalogReservation_RESERVATION_EXISTS, + v1alpha1.NodePhaseRunning, + v1alpha1.NodePhaseRunning, + v1alpha1.NodePhaseNotYetStarted, + v1alpha1.NodePhaseNotYetStarted, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + testScope := promutils.NewTestScope() + + dataStore := createInmemoryDataStore(t, testScope.NewSubScope("data_store")) + mockWorkflow := createMockWorkflow(test.currentNodePhase, test.currentDownstreamNodePhase, dataStore) + + // retrieve current node references + currentNodeSpec, ok := mockWorkflow.WorkflowSpec.Nodes[currentNodeID] + assert.Equal(t, true, ok) + + currentNodeStatus, ok := mockWorkflow.Status.NodeStatus[currentNodeID] + assert.Equal(t, true, ok) + + downstreamNodeStatus, ok := mockWorkflow.Status.NodeStatus[downstreamNodeID] + assert.Equal(t, true, ok) + + // initialize nodeExecutor + catalogClient := &catalogmocks.Client{} + catalogClient.OnGetMatch(mock.Anything, mock.Anything). + Return(pluginscatalog.NewCatalogEntry(nil, pluginscatalog.NewStatus(test.cacheStatus, nil)), nil) + + mockHandler := &nodeHandlerMocks.CacheableNode{} + mockHandler.OnIsCacheableMatch( + mock.Anything, + mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == currentNodeID }), + ).Return(test.cacheable, test.cacheSerializable, nil) + mockHandler.OnIsCacheableMatch( + mock.Anything, + mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == downstreamNodeID }), + ).Return(false, false, nil) + mockHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything). + Return(pluginscatalog.Key{Identifier: core.Identifier{Name: currentNodeID}}, nil) + mockHandler.OnFinalizeRequiredMatch(mock.Anything).Return(false) + + nodeExecutor := setupNodeExecutor(t, catalogClient, dataStore, mockHandler, testScope.NewSubScope("node_executor")) + + execContext := executors.NewExecutionContext(mockWorkflow, mockWorkflow, mockWorkflow, nil, executors.InitializeControlFlow()) + + // execute RecursiveNodeHandler + _, err := nodeExecutor.RecursiveNodeHandler(context.Background(), execContext, mockWorkflow, mockWorkflow, currentNodeSpec) + assert.NoError(t, err) + + // validate node phase transitions + assert.Equal(t, test.nextNodePhase, currentNodeStatus.Phase) + assert.Equal(t, test.nextDownstreamNodePhase, downstreamNodeStatus.Phase) + }) + } +} From 3dc1f143d583e22f5cba13f1ff6e70015a53c659 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 11 Oct 2022 22:07:33 -0500 Subject: [PATCH 24/40] fixed cache unit tets Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor_test.go | 24 +++++++++++------------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/pkg/controller/nodes/executor_test.go b/pkg/controller/nodes/executor_test.go index e6dc89ad9..88a914ac3 100644 --- a/pkg/controller/nodes/executor_test.go +++ b/pkg/controller/nodes/executor_test.go @@ -11,6 +11,7 @@ import ( "github.com/flyteorg/flyteidl/clients/go/coreutils" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/admin" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/datacatalog" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" mocks3 "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io/mocks" @@ -2490,14 +2491,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { Kind: v1alpha1.NodeKindTask, } - /*var err *v1alpha1.ExecutionError - if p == v1alpha1.NodePhaseFailing || p == v1alpha1.NodePhaseFailed { - err = &v1alpha1.ExecutionError{ExecutionError: &core.ExecutionError{Code: "test", Message: "test"}} - }*/ currentNodeStatus := &v1alpha1.NodeStatus{ Phase: currentNodePhase, - //LastAttemptStartedAt: &v1.Time{}, - //Error: err, } downstreamNodeSpec := &v1alpha1.NodeSpec{ @@ -2584,7 +2579,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { cacheable bool cacheStatus core.CatalogCacheStatus cacheSerializable bool - cacheSerializeStatus core.CatalogReservation_Status + cacheReservationOwnerID string currentNodePhase v1alpha1.NodePhase nextNodePhase v1alpha1.NodePhase currentDownstreamNodePhase v1alpha1.NodePhase @@ -2595,7 +2590,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { true, core.CatalogCacheStatus_CACHE_MISS, false, - core.CatalogReservation_RESERVATION_DISABLED, + "", v1alpha1.NodePhaseNotYetStarted, v1alpha1.NodePhaseQueued, v1alpha1.NodePhaseNotYetStarted, @@ -2606,7 +2601,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { true, core.CatalogCacheStatus_CACHE_HIT, false, - core.CatalogReservation_RESERVATION_DISABLED, + "", v1alpha1.NodePhaseNotYetStarted, v1alpha1.NodePhaseSucceeded, v1alpha1.NodePhaseNotYetStarted, @@ -2617,7 +2612,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { true, core.CatalogCacheStatus_CACHE_HIT, true, - core.CatalogReservation_RESERVATION_EXISTS, + "another-node", v1alpha1.NodePhaseQueued, v1alpha1.NodePhaseSucceeded, v1alpha1.NodePhaseNotYetStarted, @@ -2628,7 +2623,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { true, core.CatalogCacheStatus_CACHE_MISS, true, - core.CatalogReservation_RESERVATION_EXISTS, + "another-node", v1alpha1.NodePhaseQueued, v1alpha1.NodePhaseQueued, v1alpha1.NodePhaseNotYetStarted, @@ -2639,7 +2634,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { true, core.CatalogCacheStatus_CACHE_MISS, true, - core.CatalogReservation_RESERVATION_ACQUIRED, + fmt.Sprintf("%s-%d", currentNodeID, 0), v1alpha1.NodePhaseQueued, v1alpha1.NodePhaseRunning, v1alpha1.NodePhaseNotYetStarted, @@ -2650,7 +2645,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { true, core.CatalogCacheStatus_CACHE_MISS, true, - core.CatalogReservation_RESERVATION_EXISTS, + "another-node", v1alpha1.NodePhaseRunning, v1alpha1.NodePhaseRunning, v1alpha1.NodePhaseNotYetStarted, @@ -2678,6 +2673,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { catalogClient := &catalogmocks.Client{} catalogClient.OnGetMatch(mock.Anything, mock.Anything). Return(pluginscatalog.NewCatalogEntry(nil, pluginscatalog.NewStatus(test.cacheStatus, nil)), nil) + catalogClient.OnGetOrExtendReservationMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + Return(&datacatalog.Reservation{OwnerId: test.cacheReservationOwnerID}, nil) mockHandler := &nodeHandlerMocks.CacheableNode{} mockHandler.OnIsCacheableMatch( @@ -2690,6 +2687,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { ).Return(false, false, nil) mockHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything). Return(pluginscatalog.Key{Identifier: core.Identifier{Name: currentNodeID}}, nil) + mockHandler.OnHandleMatch(mock.Anything, mock.Anything).Return(handler.DoTransition(handler.TransitionTypeEphemeral, handler.PhaseInfoRunning(nil)), nil) mockHandler.OnFinalizeRequiredMatch(mock.Anything).Return(false) nodeExecutor := setupNodeExecutor(t, catalogClient, dataStore, mockHandler, testScope.NewSubScope("node_executor")) From 4093692cc0102b92b2994f49be9120dc330ba2c1 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 11 Oct 2022 22:10:19 -0500 Subject: [PATCH 25/40] fixed lint issues Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor_test.go | 54 ++++++++++++--------------- 1 file changed, 23 insertions(+), 31 deletions(-) diff --git a/pkg/controller/nodes/executor_test.go b/pkg/controller/nodes/executor_test.go index 88a914ac3..86ce69a7c 100644 --- a/pkg/controller/nodes/executor_test.go +++ b/pkg/controller/nodes/executor_test.go @@ -14,9 +14,9 @@ import ( "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/datacatalog" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/event" - mocks3 "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io/mocks" pluginscatalog "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" catalogmocks "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog/mocks" + mocks3 "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io/mocks" "github.com/flyteorg/flytepropeller/events" eventsErr "github.com/flyteorg/flytepropeller/events/errors" @@ -2485,26 +2485,6 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { taskID := taskID createMockWorkflow := func(currentNodePhase, downstreamNodePhase v1alpha1.NodePhase, dataStore *storage.DataStore) *v1alpha1.FlyteWorkflow { - currentNodeSpec := &v1alpha1.NodeSpec{ - ID: currentNodeID, - TaskRef: &taskID, - Kind: v1alpha1.NodeKindTask, - } - - currentNodeStatus := &v1alpha1.NodeStatus{ - Phase: currentNodePhase, - } - - downstreamNodeSpec := &v1alpha1.NodeSpec{ - ID: downstreamNodeID, - TaskRef: &taskID, - Kind: v1alpha1.NodeKindTask, - } - - downstreamNodeStatus := &v1alpha1.NodeStatus{ - Phase: downstreamNodePhase, - } - return &v1alpha1.FlyteWorkflow{ Tasks: map[v1alpha1.TaskID]*v1alpha1.TaskSpec{ taskID: { @@ -2513,16 +2493,28 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { }, Status: v1alpha1.WorkflowStatus{ NodeStatus: map[v1alpha1.NodeID]*v1alpha1.NodeStatus{ - currentNodeID: currentNodeStatus, - downstreamNodeID: downstreamNodeStatus, + currentNodeID: &v1alpha1.NodeStatus{ + Phase: currentNodePhase, + }, + downstreamNodeID: &v1alpha1.NodeStatus{ + Phase: downstreamNodePhase, + }, }, DataDir: "data", }, WorkflowSpec: &v1alpha1.WorkflowSpec{ ID: "wf", Nodes: map[v1alpha1.NodeID]*v1alpha1.NodeSpec{ - currentNodeID: currentNodeSpec, - downstreamNodeID: downstreamNodeSpec, + currentNodeID: &v1alpha1.NodeSpec{ + ID: currentNodeID, + TaskRef: &taskID, + Kind: v1alpha1.NodeKindTask, + }, + downstreamNodeID: &v1alpha1.NodeSpec{ + ID: downstreamNodeID, + TaskRef: &taskID, + Kind: v1alpha1.NodeKindTask, + }, }, Connections: v1alpha1.Connections{ Upstream: map[v1alpha1.NodeID][]v1alpha1.NodeID{ @@ -2678,13 +2670,13 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { mockHandler := &nodeHandlerMocks.CacheableNode{} mockHandler.OnIsCacheableMatch( - mock.Anything, - mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == currentNodeID }), - ).Return(test.cacheable, test.cacheSerializable, nil) + mock.Anything, + mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == currentNodeID }), + ).Return(test.cacheable, test.cacheSerializable, nil) mockHandler.OnIsCacheableMatch( - mock.Anything, - mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == downstreamNodeID }), - ).Return(false, false, nil) + mock.Anything, + mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == downstreamNodeID }), + ).Return(false, false, nil) mockHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything). Return(pluginscatalog.Key{Identifier: core.Identifier{Name: currentNodeID}}, nil) mockHandler.OnHandleMatch(mock.Anything, mock.Anything).Return(handler.DoTransition(handler.TransitionTypeEphemeral, handler.PhaseInfoRunning(nil)), nil) From 270396a723dc8153606f0bc8c02ecaa74e9850fc Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Wed, 19 Oct 2022 13:02:16 -0500 Subject: [PATCH 26/40] transitioning to 'Succeeded' immediately on cache hit Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 29abad110..52ecd17e1 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -534,7 +534,7 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "failed to move from queued") } - if np == v1alpha1.NodePhaseSucceeding && !h.FinalizeRequired() { + if np == v1alpha1.NodePhaseSucceeding && (!h.FinalizeRequired() || (cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT)) { logger.Infof(ctx, "Finalize not required, moving node to Succeeded") np = v1alpha1.NodePhaseSucceeded } @@ -750,7 +750,7 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut finalStatus = executors.NodeStatusTimedOut } - if np == v1alpha1.NodePhaseSucceeding && !h.FinalizeRequired() { + if np == v1alpha1.NodePhaseSucceeding && (!h.FinalizeRequired() || (cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT)) { logger.Infof(ctx, "Finalize not required, moving node to Succeeded") np = v1alpha1.NodePhaseSucceeded finalStatus = executors.NodeStatusSuccess From 0b6512092177892cf4e01834677d30bd90d21895 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 30 Mar 2023 15:25:28 -0500 Subject: [PATCH 27/40] supporting cache overwrite Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 24 ++++++++++++++++++------ 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 8eaa9505d..78ea1f96c 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -88,6 +88,7 @@ type nodeMetrics struct { catalogPutSuccessCount labeled.Counter catalogMissCount labeled.Counter catalogHitCount labeled.Counter + catalogSkipCount labeled.Counter reservationGetSuccessCount labeled.Counter reservationGetFailureCount labeled.Counter reservationReleaseSuccessCount labeled.Counter @@ -559,6 +560,11 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor if err != nil { logger.Errorf(ctx, "failed to determine if node is cacheable with err '%s'", err.Error()) return executors.NodeStatusUndefined, err + } else if cacheable && nCtx.ExecutionContext().GetExecutionConfig().OverwriteCache { + logger.Info(ctx, "execution config forced cache skip, not checking catalog") + status := catalog.NewStatus(core.CatalogCacheStatus_CACHE_SKIPPED, nil) + cacheStatus = &status + c.metrics.catalogSkipCount.Inc(ctx) } else if cacheable { entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) if err != nil { @@ -624,11 +630,11 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusQueued, nil } else if np == v1alpha1.NodePhaseSkipped { return executors.NodeStatusSuccess, nil - } else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + } /*else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { // if cache hit then immediately process downstream nodes nodeStatus.ResetDirty() return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) - } + } */ // TODO @hamersaw disable cache hit handleDownstream return executors.NodeStatusPending, nil } @@ -656,7 +662,10 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut // TODO @hamersaw - document since we already check cache in queued the first time we hit this we shouldn't check it // could potentially use nodeStatus.GetMessage() check and update on RESERVATION_EXISTS - if cacheSerializable && currentPhase == v1alpha1.NodePhaseQueued { + if cacheSerializable && nCtx.ExecutionContext().GetExecutionConfig().OverwriteCache { + status := catalog.NewStatus(core.CatalogCacheStatus_CACHE_SKIPPED, nil) + cacheStatus = &status + } else if cacheSerializable && currentPhase == v1alpha1.NodePhaseQueued { entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) if err != nil { logger.Errorf(ctx, "failed to check the catalog cache with err '%s'", err.Error()) @@ -682,7 +691,9 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } } - if cacheSerializable && (cacheStatus == nil || cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT) { + if cacheSerializable && !nCtx.ExecutionContext().GetExecutionConfig().OverwriteCache && + (cacheStatus == nil || (cacheStatus.GetCacheStatus() != core.CatalogCacheStatus_CACHE_HIT)) { + entry, err := c.GetOrExtendCatalogReservation(ctx, nCtx, cacheHandler, config.GetConfig().WorkflowReEval.Duration) if err != nil { logger.Errorf(ctx, "failed to check for catalog reservation with err '%s'", err.Error()) @@ -875,11 +886,11 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) - if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + /*if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { // if cache hit then immediately process downstream nodes nodeStatus.ResetDirty() return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) - } + }*/ // TODO @hamersaw disable cache hit handleDownstream return finalStatus, nil } @@ -1422,6 +1433,7 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora NodeInputGatherLatency: labeled.NewStopWatch("node_input_latency", "Measures the latency to aggregate inputs and check readiness of a node", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", scope), catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", scope), + catalogSkipCount: labeled.NewCounter("discovery_skip_count", "Task cached skipped in Discovery", scope), catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", scope), catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", scope), catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", scope), From cd72a0e8eb360387cec94ee99b31bc1b942abbb5 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 30 Mar 2023 16:40:39 -0500 Subject: [PATCH 28/40] fixed lint issues Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 78ea1f96c..cba490b86 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -56,6 +56,8 @@ import ( metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) +const cacheSerializedReason = "waiting on serialized cache" + type nodeMetrics struct { Scope promutils.Scope FailureDuration labeled.StopWatch @@ -630,11 +632,11 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusQueued, nil } else if np == v1alpha1.NodePhaseSkipped { return executors.NodeStatusSuccess, nil - } /*else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + } else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { // if cache hit then immediately process downstream nodes nodeStatus.ResetDirty() return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) - } */ // TODO @hamersaw disable cache hit handleDownstream + } return executors.NodeStatusPending, nil } @@ -660,12 +662,14 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut return executors.NodeStatusUndefined, err } - // TODO @hamersaw - document since we already check cache in queued the first time we hit this we shouldn't check it - // could potentially use nodeStatus.GetMessage() check and update on RESERVATION_EXISTS if cacheSerializable && nCtx.ExecutionContext().GetExecutionConfig().OverwriteCache { status := catalog.NewStatus(core.CatalogCacheStatus_CACHE_SKIPPED, nil) cacheStatus = &status - } else if cacheSerializable && currentPhase == v1alpha1.NodePhaseQueued { + } else if cacheSerializable && currentPhase == v1alpha1.NodePhaseQueued && nodeStatus.GetMessage() == cacheSerializedReason { + // since we already check the cache before transitioning to Phase Queued we only need to check it again if + // the cache is serialized and that causes the node to stay in the Queued phase. the easiest way to detect + // this is verifying the NodeStatus Reason is what we set it during cache serialization. + entry, err := c.CheckCatalogCache(ctx, nCtx, cacheHandler) if err != nil { logger.Errorf(ctx, "failed to check the catalog cache with err '%s'", err.Error()) @@ -706,7 +710,7 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut logger.Infof(ctx, "acquired cache reservation") } else if status == core.CatalogReservation_RESERVATION_EXISTS { // if reservation is held by another owner we stay in the queued phase - p = handler.PhaseInfoQueued("waiting on serialized cache", nil) + p = handler.PhaseInfoQueued(cacheSerializedReason, nil) } } } @@ -886,11 +890,11 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) - /*if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { + if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { // if cache hit then immediately process downstream nodes nodeStatus.ResetDirty() return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) - }*/ // TODO @hamersaw disable cache hit handleDownstream + } return finalStatus, nil } From dcaefdc89eb52c02e611f0a269cd0573427477f1 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 3 Apr 2023 13:47:06 -0500 Subject: [PATCH 29/40] removed automatic downstream on cache hit Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 15 ++------------- pkg/controller/nodes/executor_test.go | 7 ++++--- 2 files changed, 6 insertions(+), 16 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index cba490b86..5367f0a4f 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -632,16 +632,12 @@ func (c *nodeExecutor) handleNotYetStartedNode(ctx context.Context, dag executor return executors.NodeStatusQueued, nil } else if np == v1alpha1.NodePhaseSkipped { return executors.NodeStatusSuccess, nil - } else if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - // if cache hit then immediately process downstream nodes - nodeStatus.ResetDirty() - return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) } return executors.NodeStatusPending, nil } -func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag executors.DAGStructure, nCtx *nodeExecContext, h handler.Node) (executors.NodeStatus, error) { +func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, nCtx *nodeExecContext, h handler.Node) (executors.NodeStatus, error) { nodeStatus := nCtx.NodeStatus() currentPhase := nodeStatus.GetPhase() p := handler.PhaseInfoUndefined @@ -889,13 +885,6 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, dag execut } UpdateNodeStatus(np, p, nCtx.nsm, nodeStatus) - - if cacheStatus != nil && cacheStatus.GetCacheStatus() == core.CatalogCacheStatus_CACHE_HIT { - // if cache hit then immediately process downstream nodes - nodeStatus.ResetDirty() - return c.handleDownstream(ctx, nCtx.ExecutionContext(), dag, nCtx.ContextualNodeLookup(), nCtx.Node()) - } - return finalStatus, nil } @@ -999,7 +988,7 @@ func (c *nodeExecutor) handleNode(ctx context.Context, dag executors.DAGStructur return executors.NodeStatusFailed(nodeStatus.GetExecutionError()), nil } - return c.handleQueuedOrRunningNode(ctx, dag, nCtx, h) + return c.handleQueuedOrRunningNode(ctx, nCtx, h) } // The space search for the next node to execute is implemented like a DFS algorithm. handleDownstream visits all the nodes downstream from diff --git a/pkg/controller/nodes/executor_test.go b/pkg/controller/nodes/executor_test.go index 7c6f57ef0..3604c50ff 100644 --- a/pkg/controller/nodes/executor_test.go +++ b/pkg/controller/nodes/executor_test.go @@ -2604,7 +2604,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { Status: v1alpha1.WorkflowStatus{ NodeStatus: map[v1alpha1.NodeID]*v1alpha1.NodeStatus{ currentNodeID: &v1alpha1.NodeStatus{ - Phase: currentNodePhase, + Phase: currentNodePhase, + Message: cacheSerializedReason, }, downstreamNodeID: &v1alpha1.NodeStatus{ Phase: downstreamNodePhase, @@ -2707,7 +2708,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { v1alpha1.NodePhaseNotYetStarted, v1alpha1.NodePhaseSucceeded, v1alpha1.NodePhaseNotYetStarted, - v1alpha1.NodePhaseQueued, + v1alpha1.NodePhaseNotYetStarted, }, { "Queued->CacheHit->Success", @@ -2718,7 +2719,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { v1alpha1.NodePhaseQueued, v1alpha1.NodePhaseSucceeded, v1alpha1.NodePhaseNotYetStarted, - v1alpha1.NodePhaseQueued, + v1alpha1.NodePhaseNotYetStarted, }, { "Queued->CacheMiss->Queued", From 5ea74d8172d79bdb11bf36adcbcbb50353803df0 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 3 Aug 2023 10:04:36 -0500 Subject: [PATCH 30/40] bumping boilerplate support tools to go 1.19 to fix generate Signed-off-by: Daniel Rammer --- boilerplate/flyte/golang_support_tools/go.mod | 15 +- boilerplate/flyte/golang_support_tools/go.sum | 52 +--- .../nodes/dynamic/mocks/task_node_handler.go | 18 +- .../mocks/cacheable_node_handler.go | 272 ++++++++++++++++++ 4 files changed, 301 insertions(+), 56 deletions(-) create mode 100644 pkg/controller/nodes/interfaces/mocks/cacheable_node_handler.go diff --git a/boilerplate/flyte/golang_support_tools/go.mod b/boilerplate/flyte/golang_support_tools/go.mod index dbf94f411..d60582796 100644 --- a/boilerplate/flyte/golang_support_tools/go.mod +++ b/boilerplate/flyte/golang_support_tools/go.mod @@ -1,6 +1,6 @@ module github.com/flyteorg/boilerplate -go 1.17 +go 1.19 require ( github.com/EngHabu/mockery v0.0.0-20220405200825-3f76291311cf @@ -163,16 +163,15 @@ require ( github.com/ultraware/whitespace v0.0.4 // indirect github.com/uudashr/gocognit v1.0.1 // indirect go.opencensus.io v0.22.6 // indirect - golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 // indirect + golang.org/x/crypto v0.11.0 // indirect golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5 // indirect - golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3 // indirect - golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f // indirect + golang.org/x/mod v0.12.0 // indirect + golang.org/x/net v0.12.0 // indirect golang.org/x/oauth2 v0.0.0-20210126194326-f9ce19ea3013 // indirect - golang.org/x/sys v0.0.0-20211019181941-9d821ace8654 // indirect - golang.org/x/text v0.3.7 // indirect + golang.org/x/sys v0.10.0 // indirect + golang.org/x/text v0.11.0 // indirect golang.org/x/time v0.0.0-20201208040808-7e3f01d25324 // indirect - golang.org/x/tools v0.1.10 // indirect - golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect + golang.org/x/tools v0.11.1 // indirect google.golang.org/api v0.38.0 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20210126160654-44e461bb6506 // indirect diff --git a/boilerplate/flyte/golang_support_tools/go.sum b/boilerplate/flyte/golang_support_tools/go.sum index 02895fb57..c9b663bf4 100644 --- a/boilerplate/flyte/golang_support_tools/go.sum +++ b/boilerplate/flyte/golang_support_tools/go.sum @@ -51,27 +51,19 @@ github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v0.3.0 h1:Px2UA+2RvSSvv+RvJ github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v0.3.0/go.mod h1:tPaiy8S5bQ+S5sOiDlINkp7+Ef339+Nz5L5XO+cnOHo= github.com/Azure/go-autorest v14.2.0+incompatible h1:V5VMDjClD3GiElqLWO7mz2MxNAK/vTfRHdAubSIPRgs= github.com/Azure/go-autorest v14.2.0+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24= -github.com/Azure/go-autorest/autorest v0.9.0/go.mod h1:xyHB1BMZT0cuDHU7I0+g046+BFDTQ8rEZB0s4Yfa6bI= github.com/Azure/go-autorest/autorest v0.11.12/go.mod h1:eipySxLmqSyC5s5k1CLupqet0PSENBEDP93LQ9a8QYw= github.com/Azure/go-autorest/autorest v0.11.17 h1:2zCdHwNgRH+St1J+ZMf66xI8aLr/5KMy+wWLH97zwYM= github.com/Azure/go-autorest/autorest v0.11.17/go.mod h1:eipySxLmqSyC5s5k1CLupqet0PSENBEDP93LQ9a8QYw= -github.com/Azure/go-autorest/autorest/adal v0.5.0/go.mod h1:8Z9fGy2MpX0PvDjB1pEgQTmVqjGhiHBW7RJJEciWzS0= github.com/Azure/go-autorest/autorest/adal v0.9.5/go.mod h1:B7KF7jKIeC9Mct5spmyCB/A8CG/sEz1vwIRGv/bbw7A= github.com/Azure/go-autorest/autorest/adal v0.9.10 h1:r6fZHMaHD8B6LDCn0o5vyBFHIHrM6Ywwx7mb49lPItI= github.com/Azure/go-autorest/autorest/adal v0.9.10/go.mod h1:B7KF7jKIeC9Mct5spmyCB/A8CG/sEz1vwIRGv/bbw7A= -github.com/Azure/go-autorest/autorest/date v0.1.0/go.mod h1:plvfp3oPSKwf2DNjlBjWF/7vwR+cUD/ELuzDCXwHUVA= github.com/Azure/go-autorest/autorest/date v0.3.0 h1:7gUk1U5M/CQbp9WoqinNzJar+8KY+LPI6wiWrP/myHw= github.com/Azure/go-autorest/autorest/date v0.3.0/go.mod h1:BI0uouVdmngYNUzGWeSYnokU+TrmwEsOqdt8Y6sso74= -github.com/Azure/go-autorest/autorest/mocks v0.1.0/go.mod h1:OTyCOPRA2IgIlWxVYxBee2F5Gr4kF2zd2J5cFRaIDN0= -github.com/Azure/go-autorest/autorest/mocks v0.2.0/go.mod h1:OTyCOPRA2IgIlWxVYxBee2F5Gr4kF2zd2J5cFRaIDN0= github.com/Azure/go-autorest/autorest/mocks v0.4.1 h1:K0laFcLE6VLTOwNgSxaGbUcLPuGXlNkbVvq4cW4nIHk= github.com/Azure/go-autorest/autorest/mocks v0.4.1/go.mod h1:LTp+uSrOhSkaKrUy935gNZuuIPPVsHlr9DSOxSayd+k= github.com/Azure/go-autorest/autorest/to v0.4.0 h1:oXVqrxakqqV1UZdSazDOPOLvOIz+XA683u8EctwboHk= -github.com/Azure/go-autorest/autorest/to v0.4.0/go.mod h1:fE8iZBn7LQR7zH/9XU2NcPR4o9jEImooCeWJcYV/zLE= -github.com/Azure/go-autorest/logger v0.1.0/go.mod h1:oExouG+K6PryycPJfVSxi/koC6LSNgds39diKLz7Vrc= github.com/Azure/go-autorest/logger v0.2.0 h1:e4RVHVZKC5p6UANLJHkM4OfR1UKZPj8Wt8Pcx+3oqrE= github.com/Azure/go-autorest/logger v0.2.0/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZmbF5NWuPV8+WeEW8= -github.com/Azure/go-autorest/tracing v0.5.0/go.mod h1:r/s2XiOKccPW3HrqB+W0TQzfbtp2fGCgRFtBroKn4Dk= github.com/Azure/go-autorest/tracing v0.6.0 h1:TYi4+3m5t6K48TGI9AUdb+IzbnSxvnvUMfuitfgcfuo= github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= @@ -123,12 +115,10 @@ github.com/ashanbrown/forbidigo v1.1.0/go.mod h1:vVW7PEdqEFqapJe95xHkTfB1+XvZXBF github.com/ashanbrown/makezero v0.0.0-20201205152432-7b7cdbb3025a h1:/U9tbJzDRof4fOR51vwzWdIBsIH6R2yU0KG1MBRM2Js= github.com/ashanbrown/makezero v0.0.0-20201205152432-7b7cdbb3025a/go.mod h1:oG9Dnez7/ESBqc4EdrdNlryeo7d0KcW1ftXHm7nU/UU= github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= -github.com/aws/aws-sdk-go v1.23.4/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.37.1 h1:BTHmuN+gzhxkvU9sac2tZvaY0gV9ihbHw+KxZOecYvY= github.com/aws/aws-sdk-go v1.37.1/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= -github.com/benlaurie/objecthash v0.0.0-20180202135721-d1e3d6079fc1/go.mod h1:jvdWlw8vowVGnZqSDC7yhPd7AifQeQbRDkZcQXV2nRg= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -149,7 +139,6 @@ github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XL github.com/charithe/durationcheck v0.0.6 h1:Tsy7EppNow2pDC0jN7Hsmcb6mHd71ZbI1vFissRBtc0= github.com/charithe/durationcheck v0.0.6/go.mod h1:SSbRIBVfMjCi/kEB6K65XEA83D6prSM8ap1UCpNKtgg= github.com/cheekybits/is v0.0.0-20150225183255-68e9c0620927 h1:SKI1/fuSdodxmNNyVBR8d7X/HuLnRpvvFO0AgyQk764= -github.com/cheekybits/is v0.0.0-20150225183255-68e9c0620927/go.mod h1:h/aW8ynjgkuj+NQRlZcDbAbM1ORAbXjXX77sX7T289U= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -259,7 +248,6 @@ github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-test/deep v1.0.7/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= github.com/go-toolsmith/astcast v1.0.0 h1:JojxlmI6STnFVG9yOImLeGREv8W2ocNUM+iOhR6jE7g= github.com/go-toolsmith/astcast v1.0.0/go.mod h1:mt2OdQTeAQcY4DQgPSArJjHCcOwlX+Wl/kwN+LbLGQ4= github.com/go-toolsmith/astcopy v1.0.0 h1:OMgl1b1MEpjFQ1m5ztEO06rz5CUd3oBv9RF7+DyvdG8= @@ -374,7 +362,6 @@ github.com/google/pprof v0.0.0-20200905233945-acf8798be1f7/go.mod h1:ZgVRPoUq/hf github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20201218002935-b9804c9f04c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/readahead v0.0.0-20161222183148-eaceba169032/go.mod h1:qYysrqQXuV4tzsizt4oOQ6mrBZQ0xnQXP3ylXX8Jk5Y= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v0.0.0-20161128191214-064e2069ce9c/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -431,7 +418,6 @@ github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09 github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= @@ -584,7 +570,6 @@ github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxzi github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/nbutton23/zxcvbn-go v0.0.0-20201221231540-e56b841a3c88 h1:o+O3Cd1HO9CTgxE3/C8p5I5Y4C0yYWbF8d4IkfOLtcQ= github.com/nbutton23/zxcvbn-go v0.0.0-20201221231540-e56b841a3c88/go.mod h1:KSVJerMDfblTH7p5MZaTt+8zaT2iEk3AkVb9PQdZuE8= -github.com/ncw/swift v1.0.49/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= github.com/ncw/swift v1.0.53 h1:luHjjTNtekIEvHg5KdAFIBaH7bWfNkefwFnpDffSIks= github.com/ncw/swift v1.0.53/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= @@ -642,7 +627,6 @@ github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= -github.com/pkg/sftp v1.10.0/go.mod h1:NxmoDg/QLVWluQDUYG7XBZTLUpKeFa8e3aMf1BfjyHk= github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pmezard/go-difflib v0.0.0-20151028094244-d8ed2627bdf0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -650,7 +634,6 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/polyfloyd/go-errorlint v0.0.0-20201127212506-19bd8db6546f h1:xAw10KgJqG5NJDfmRqJ05Z0IFblKumjtMeyiOLxj3+4= github.com/polyfloyd/go-errorlint v0.0.0-20201127212506-19bd8db6546f/go.mod h1:wi9BfjxjF/bwiZ701TzmfKu6UKC357IOAtNr0Td0Lvw= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= -github.com/pquerna/ffjson v0.0.0-20190813045741-dac163c6c0a9/go.mod h1:YARuvh7BUWHNhzDq2OM5tzR2RiCcN2D7sapiKyCel/M= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= @@ -719,7 +702,6 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/sirupsen/logrus v1.8.0 h1:nfhvjKcUMhBMVqbKHJlk5RPrrfYr/NMo3692g0dwfWU= github.com/sirupsen/logrus v1.8.0/go.mod h1:4GuYW9TZmE769R5STWrRakJc4UqQ3+QQ95fyz7ENv1A= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= @@ -742,7 +724,6 @@ github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkU github.com/spf13/cast v1.3.1 h1:nFm6S0SMdyzrzcmThSipiEubIDy8WEXKNZ0UOgiRpng= github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= -github.com/spf13/cobra v1.1.1/go.mod h1:WnodtKOvamDL/PwE2M4iKs8aMDBZ5Q5klgD3qfVJQMI= github.com/spf13/cobra v1.1.3 h1:xghbfqPkxzxP3C/f3n5DdpAbdKLj4ZE4BWQI362l53M= github.com/spf13/cobra v1.1.3/go.mod h1:pGADOWyqRD/YMrPZigI/zbliZ2wVD/23d+is3pSWzOo= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= @@ -807,7 +788,6 @@ github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= @@ -841,9 +821,8 @@ golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= +golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -877,9 +856,8 @@ golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3 h1:kQgndtyPBW/JIYERgdxfwMYh3AVStj88WQTlNDi2a+o= -golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3/go.mod h1:3p9vT2HGsQu2K1YbXdKPJLVgG5VJdoTa1poYQBtP1AY= +golang.org/x/mod v0.12.0 h1:rmsUpXtvNzj340zd98LZ4KntptpfRHwpFOHG188oHXc= +golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -925,11 +903,10 @@ golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f h1:OfiFi4JbukWwe3lzw+xunroH1mnC1e2Gy5cxNJApiSY= -golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.12.0 h1:cfawfvKITfUsFCeJIHJrbSxpeu/E81khclypR0GVT50= +golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -951,7 +928,7 @@ golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1010,10 +987,8 @@ golang.org/x/sys v0.0.0-20201214210602-f9fddec55a1e/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211019181941-9d821ace8654 h1:id054HUawV2/6IGm2IV8KZQjqtwAOo2CYlOToYqa0d0= -golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= +golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= +golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1022,8 +997,9 @@ golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.11.0 h1:LAntKIrcmeSKERyiOh0XMV39LXS8IE9UL2yP7+f5ij4= +golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1125,8 +1101,8 @@ golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= -golang.org/x/tools v0.1.10 h1:QjFRCZxdOhBJ/UNgnBZLbNV13DlbnK0quyivTnXJM20= -golang.org/x/tools v0.1.10/go.mod h1:Uh6Zz+xoGYZom868N8YTex3t7RhtHDBrE8Gzo9bV56E= +golang.org/x/tools v0.11.1 h1:ojD5zOW8+7dOGzdnNgersm8aPfcDjhMp12UfG93NIMc= +golang.org/x/tools v0.11.1/go.mod h1:anzJrxPjNtfgiYQYirP2CPGzGLxrH2u2QBhn6Bf3qY8= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1255,7 +1231,6 @@ gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.62.0 h1:duBzk771uxoUuOlyRLkHsygud9+5lrlGjdFBb4mSKDU= gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/kothar/go-backblaze.v0 v0.0.0-20190520213052-702d4e7eb465/go.mod h1:zJ2QpyDCYo1KvLXlmdnFlQAyF/Qfth0fB8239Qg7BIE= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= @@ -1283,7 +1258,6 @@ honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9 honnef.co/go/tools v0.1.2 h1:SMdYLJl312RXuxXziCCHhRsp/tvct9cGKey0yv95tZM= honnef.co/go/tools v0.1.2/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/api v0.0.0-20210217171935-8e2decd92398/go.mod h1:60tmSUpHxGPFerNHbo/ayI2lKxvtrhbxFyXuEIWJd78= -k8s.io/api v0.20.2/go.mod h1:d7n6Ehyzx+S+cE3VhTGfVNNqtGc/oL9DCdYYahlurV8= k8s.io/apimachinery v0.0.0-20210217011835-527a61b4dffe/go.mod h1:Z7ps/g0rjlTeMstYrMOUttJfT2Gg34DEaG/f2PYLCWY= k8s.io/apimachinery v0.20.2 h1:hFx6Sbt1oG0n6DZ+g4bFt5f6BoMkOjKWsQFu077M3Vg= k8s.io/apimachinery v0.20.2/go.mod h1:WlLqWAHZGg07AeltaI0MV5uk1Omp8xaN0JGLY6gkRpU= diff --git a/pkg/controller/nodes/dynamic/mocks/task_node_handler.go b/pkg/controller/nodes/dynamic/mocks/task_node_handler.go index 5f14124c5..eeba97048 100644 --- a/pkg/controller/nodes/dynamic/mocks/task_node_handler.go +++ b/pkg/controller/nodes/dynamic/mocks/task_node_handler.go @@ -129,7 +129,7 @@ func (_m TaskNodeHandler_GetCatalogKey) Return(_a0 catalog.Key, _a1 error) *Task return &TaskNodeHandler_GetCatalogKey{Call: _m.Call.Return(_a0, _a1)} } -func (_m *TaskNodeHandler) OnGetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) *TaskNodeHandler_GetCatalogKey { +func (_m *TaskNodeHandler) OnGetCatalogKey(ctx context.Context, executionContext interfaces.NodeExecutionContext) *TaskNodeHandler_GetCatalogKey { c_call := _m.On("GetCatalogKey", ctx, executionContext) return &TaskNodeHandler_GetCatalogKey{Call: c_call} } @@ -140,18 +140,18 @@ func (_m *TaskNodeHandler) OnGetCatalogKeyMatch(matchers ...interface{}) *TaskNo } // GetCatalogKey provides a mock function with given fields: ctx, executionContext -func (_m *TaskNodeHandler) GetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) (catalog.Key, error) { +func (_m *TaskNodeHandler) GetCatalogKey(ctx context.Context, executionContext interfaces.NodeExecutionContext) (catalog.Key, error) { ret := _m.Called(ctx, executionContext) var r0 catalog.Key - if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) catalog.Key); ok { + if rf, ok := ret.Get(0).(func(context.Context, interfaces.NodeExecutionContext) catalog.Key); ok { r0 = rf(ctx, executionContext) } else { r0 = ret.Get(0).(catalog.Key) } var r1 error - if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) error); ok { + if rf, ok := ret.Get(1).(func(context.Context, interfaces.NodeExecutionContext) error); ok { r1 = rf(ctx, executionContext) } else { r1 = ret.Error(1) @@ -207,7 +207,7 @@ func (_m TaskNodeHandler_IsCacheable) Return(_a0 bool, _a1 bool, _a2 error) *Tas return &TaskNodeHandler_IsCacheable{Call: _m.Call.Return(_a0, _a1, _a2)} } -func (_m *TaskNodeHandler) OnIsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) *TaskNodeHandler_IsCacheable { +func (_m *TaskNodeHandler) OnIsCacheable(ctx context.Context, executionContext interfaces.NodeExecutionContext) *TaskNodeHandler_IsCacheable { c_call := _m.On("IsCacheable", ctx, executionContext) return &TaskNodeHandler_IsCacheable{Call: c_call} } @@ -218,25 +218,25 @@ func (_m *TaskNodeHandler) OnIsCacheableMatch(matchers ...interface{}) *TaskNode } // IsCacheable provides a mock function with given fields: ctx, executionContext -func (_m *TaskNodeHandler) IsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) (bool, bool, error) { +func (_m *TaskNodeHandler) IsCacheable(ctx context.Context, executionContext interfaces.NodeExecutionContext) (bool, bool, error) { ret := _m.Called(ctx, executionContext) var r0 bool - if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) bool); ok { + if rf, ok := ret.Get(0).(func(context.Context, interfaces.NodeExecutionContext) bool); ok { r0 = rf(ctx, executionContext) } else { r0 = ret.Get(0).(bool) } var r1 bool - if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) bool); ok { + if rf, ok := ret.Get(1).(func(context.Context, interfaces.NodeExecutionContext) bool); ok { r1 = rf(ctx, executionContext) } else { r1 = ret.Get(1).(bool) } var r2 error - if rf, ok := ret.Get(2).(func(context.Context, handler.NodeExecutionContext) error); ok { + if rf, ok := ret.Get(2).(func(context.Context, interfaces.NodeExecutionContext) error); ok { r2 = rf(ctx, executionContext) } else { r2 = ret.Error(2) diff --git a/pkg/controller/nodes/interfaces/mocks/cacheable_node_handler.go b/pkg/controller/nodes/interfaces/mocks/cacheable_node_handler.go new file mode 100644 index 000000000..39456010b --- /dev/null +++ b/pkg/controller/nodes/interfaces/mocks/cacheable_node_handler.go @@ -0,0 +1,272 @@ +// Code generated by mockery v1.0.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + catalog "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" + + handler "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" + + interfaces "github.com/flyteorg/flytepropeller/pkg/controller/nodes/interfaces" + + mock "github.com/stretchr/testify/mock" +) + +// CacheableNodeHandler is an autogenerated mock type for the CacheableNodeHandler type +type CacheableNodeHandler struct { + mock.Mock +} + +type CacheableNodeHandler_Abort struct { + *mock.Call +} + +func (_m CacheableNodeHandler_Abort) Return(_a0 error) *CacheableNodeHandler_Abort { + return &CacheableNodeHandler_Abort{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNodeHandler) OnAbort(ctx context.Context, executionContext interfaces.NodeExecutionContext, reason string) *CacheableNodeHandler_Abort { + c_call := _m.On("Abort", ctx, executionContext, reason) + return &CacheableNodeHandler_Abort{Call: c_call} +} + +func (_m *CacheableNodeHandler) OnAbortMatch(matchers ...interface{}) *CacheableNodeHandler_Abort { + c_call := _m.On("Abort", matchers...) + return &CacheableNodeHandler_Abort{Call: c_call} +} + +// Abort provides a mock function with given fields: ctx, executionContext, reason +func (_m *CacheableNodeHandler) Abort(ctx context.Context, executionContext interfaces.NodeExecutionContext, reason string) error { + ret := _m.Called(ctx, executionContext, reason) + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, interfaces.NodeExecutionContext, string) error); ok { + r0 = rf(ctx, executionContext, reason) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +type CacheableNodeHandler_Finalize struct { + *mock.Call +} + +func (_m CacheableNodeHandler_Finalize) Return(_a0 error) *CacheableNodeHandler_Finalize { + return &CacheableNodeHandler_Finalize{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNodeHandler) OnFinalize(ctx context.Context, executionContext interfaces.NodeExecutionContext) *CacheableNodeHandler_Finalize { + c_call := _m.On("Finalize", ctx, executionContext) + return &CacheableNodeHandler_Finalize{Call: c_call} +} + +func (_m *CacheableNodeHandler) OnFinalizeMatch(matchers ...interface{}) *CacheableNodeHandler_Finalize { + c_call := _m.On("Finalize", matchers...) + return &CacheableNodeHandler_Finalize{Call: c_call} +} + +// Finalize provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNodeHandler) Finalize(ctx context.Context, executionContext interfaces.NodeExecutionContext) error { + ret := _m.Called(ctx, executionContext) + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, interfaces.NodeExecutionContext) error); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +type CacheableNodeHandler_FinalizeRequired struct { + *mock.Call +} + +func (_m CacheableNodeHandler_FinalizeRequired) Return(_a0 bool) *CacheableNodeHandler_FinalizeRequired { + return &CacheableNodeHandler_FinalizeRequired{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNodeHandler) OnFinalizeRequired() *CacheableNodeHandler_FinalizeRequired { + c_call := _m.On("FinalizeRequired") + return &CacheableNodeHandler_FinalizeRequired{Call: c_call} +} + +func (_m *CacheableNodeHandler) OnFinalizeRequiredMatch(matchers ...interface{}) *CacheableNodeHandler_FinalizeRequired { + c_call := _m.On("FinalizeRequired", matchers...) + return &CacheableNodeHandler_FinalizeRequired{Call: c_call} +} + +// FinalizeRequired provides a mock function with given fields: +func (_m *CacheableNodeHandler) FinalizeRequired() bool { + ret := _m.Called() + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +type CacheableNodeHandler_GetCatalogKey struct { + *mock.Call +} + +func (_m CacheableNodeHandler_GetCatalogKey) Return(_a0 catalog.Key, _a1 error) *CacheableNodeHandler_GetCatalogKey { + return &CacheableNodeHandler_GetCatalogKey{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *CacheableNodeHandler) OnGetCatalogKey(ctx context.Context, executionContext interfaces.NodeExecutionContext) *CacheableNodeHandler_GetCatalogKey { + c_call := _m.On("GetCatalogKey", ctx, executionContext) + return &CacheableNodeHandler_GetCatalogKey{Call: c_call} +} + +func (_m *CacheableNodeHandler) OnGetCatalogKeyMatch(matchers ...interface{}) *CacheableNodeHandler_GetCatalogKey { + c_call := _m.On("GetCatalogKey", matchers...) + return &CacheableNodeHandler_GetCatalogKey{Call: c_call} +} + +// GetCatalogKey provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNodeHandler) GetCatalogKey(ctx context.Context, executionContext interfaces.NodeExecutionContext) (catalog.Key, error) { + ret := _m.Called(ctx, executionContext) + + var r0 catalog.Key + if rf, ok := ret.Get(0).(func(context.Context, interfaces.NodeExecutionContext) catalog.Key); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(catalog.Key) + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, interfaces.NodeExecutionContext) error); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type CacheableNodeHandler_Handle struct { + *mock.Call +} + +func (_m CacheableNodeHandler_Handle) Return(_a0 handler.Transition, _a1 error) *CacheableNodeHandler_Handle { + return &CacheableNodeHandler_Handle{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *CacheableNodeHandler) OnHandle(ctx context.Context, executionContext interfaces.NodeExecutionContext) *CacheableNodeHandler_Handle { + c_call := _m.On("Handle", ctx, executionContext) + return &CacheableNodeHandler_Handle{Call: c_call} +} + +func (_m *CacheableNodeHandler) OnHandleMatch(matchers ...interface{}) *CacheableNodeHandler_Handle { + c_call := _m.On("Handle", matchers...) + return &CacheableNodeHandler_Handle{Call: c_call} +} + +// Handle provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNodeHandler) Handle(ctx context.Context, executionContext interfaces.NodeExecutionContext) (handler.Transition, error) { + ret := _m.Called(ctx, executionContext) + + var r0 handler.Transition + if rf, ok := ret.Get(0).(func(context.Context, interfaces.NodeExecutionContext) handler.Transition); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(handler.Transition) + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, interfaces.NodeExecutionContext) error); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type CacheableNodeHandler_IsCacheable struct { + *mock.Call +} + +func (_m CacheableNodeHandler_IsCacheable) Return(_a0 bool, _a1 bool, _a2 error) *CacheableNodeHandler_IsCacheable { + return &CacheableNodeHandler_IsCacheable{Call: _m.Call.Return(_a0, _a1, _a2)} +} + +func (_m *CacheableNodeHandler) OnIsCacheable(ctx context.Context, executionContext interfaces.NodeExecutionContext) *CacheableNodeHandler_IsCacheable { + c_call := _m.On("IsCacheable", ctx, executionContext) + return &CacheableNodeHandler_IsCacheable{Call: c_call} +} + +func (_m *CacheableNodeHandler) OnIsCacheableMatch(matchers ...interface{}) *CacheableNodeHandler_IsCacheable { + c_call := _m.On("IsCacheable", matchers...) + return &CacheableNodeHandler_IsCacheable{Call: c_call} +} + +// IsCacheable provides a mock function with given fields: ctx, executionContext +func (_m *CacheableNodeHandler) IsCacheable(ctx context.Context, executionContext interfaces.NodeExecutionContext) (bool, bool, error) { + ret := _m.Called(ctx, executionContext) + + var r0 bool + if rf, ok := ret.Get(0).(func(context.Context, interfaces.NodeExecutionContext) bool); ok { + r0 = rf(ctx, executionContext) + } else { + r0 = ret.Get(0).(bool) + } + + var r1 bool + if rf, ok := ret.Get(1).(func(context.Context, interfaces.NodeExecutionContext) bool); ok { + r1 = rf(ctx, executionContext) + } else { + r1 = ret.Get(1).(bool) + } + + var r2 error + if rf, ok := ret.Get(2).(func(context.Context, interfaces.NodeExecutionContext) error); ok { + r2 = rf(ctx, executionContext) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 +} + +type CacheableNodeHandler_Setup struct { + *mock.Call +} + +func (_m CacheableNodeHandler_Setup) Return(_a0 error) *CacheableNodeHandler_Setup { + return &CacheableNodeHandler_Setup{Call: _m.Call.Return(_a0)} +} + +func (_m *CacheableNodeHandler) OnSetup(ctx context.Context, setupContext interfaces.SetupContext) *CacheableNodeHandler_Setup { + c_call := _m.On("Setup", ctx, setupContext) + return &CacheableNodeHandler_Setup{Call: c_call} +} + +func (_m *CacheableNodeHandler) OnSetupMatch(matchers ...interface{}) *CacheableNodeHandler_Setup { + c_call := _m.On("Setup", matchers...) + return &CacheableNodeHandler_Setup{Call: c_call} +} + +// Setup provides a mock function with given fields: ctx, setupContext +func (_m *CacheableNodeHandler) Setup(ctx context.Context, setupContext interfaces.SetupContext) error { + ret := _m.Called(ctx, setupContext) + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, interfaces.SetupContext) error); ok { + r0 = rf(ctx, setupContext) + } else { + r0 = ret.Error(0) + } + + return r0 +} From cabfc58a08e35ca0c5a84c8dd577908da65b9e9b Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 3 Aug 2023 16:08:58 -0500 Subject: [PATCH 31/40] fixed unit tests and linter Signed-off-by: Daniel Rammer --- pkg/controller/nodes/array/handler_test.go | 2 +- pkg/controller/nodes/cache_test.go | 115 +++++--- pkg/controller/nodes/dynamic/handler_test.go | 13 +- pkg/controller/nodes/executor.go | 40 +-- pkg/controller/nodes/executor_test.go | 22 +- .../nodes/interfaces/mocks/cacheable_node.go | 270 ------------------ 6 files changed, 108 insertions(+), 354 deletions(-) delete mode 100644 pkg/controller/nodes/interfaces/mocks/cacheable_node.go diff --git a/pkg/controller/nodes/array/handler_test.go b/pkg/controller/nodes/array/handler_test.go index f3e6f8bd1..3a5f84965 100644 --- a/pkg/controller/nodes/array/handler_test.go +++ b/pkg/controller/nodes/array/handler_test.go @@ -13,13 +13,13 @@ import ( "github.com/flyteorg/flytepropeller/pkg/controller/config" execmocks "github.com/flyteorg/flytepropeller/pkg/controller/executors/mocks" "github.com/flyteorg/flytepropeller/pkg/controller/nodes" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/catalog" gatemocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/gate/mocks" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/interfaces" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/interfaces/mocks" recoverymocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/recovery/mocks" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/task/catalog" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/core" pluginmocks "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/io/mocks" diff --git a/pkg/controller/nodes/cache_test.go b/pkg/controller/nodes/cache_test.go index a08f549d6..ab77ff754 100644 --- a/pkg/controller/nodes/cache_test.go +++ b/pkg/controller/nodes/cache_test.go @@ -10,21 +10,19 @@ import ( "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyteidl/gen/pb-go/flyteidl/datacatalog" - eventsmocks "github.com/flyteorg/flytepropeller/events/mocks" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/mocks" - "github.com/flyteorg/flytepropeller/pkg/controller/config" executorsmocks "github.com/flyteorg/flytepropeller/pkg/controller/executors/mocks" "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" - handlermocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler/mocks" - recoverymocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/recovery/mocks" - "github.com/flyteorg/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" + "github.com/flyteorg/flytepropeller/pkg/controller/nodes/interfaces" + interfacesmocks "github.com/flyteorg/flytepropeller/pkg/controller/nodes/interfaces/mocks" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" catalogmocks "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog/mocks" "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flytestdlib/promutils" + "github.com/flyteorg/flytestdlib/promutils/labeled" "github.com/flyteorg/flytestdlib/storage" "k8s.io/apimachinery/pkg/types" @@ -70,7 +68,7 @@ func setupCacheableNodeExecutionContext(dataStore *storage.DataStore, taskTempla mockExecutionContext := &executorsmocks.ExecutionContext{} mockExecutionContext.OnGetParentInfoMatch(mock.Anything).Return(mockParentInfo) - mockNodeExecutionMetadata := &handlermocks.NodeExecutionMetadata{} + mockNodeExecutionMetadata := &interfacesmocks.NodeExecutionMetadata{} mockNodeExecutionMetadata.OnGetOwnerID().Return( types.NamespacedName{ Name: parentUniqueID, @@ -82,7 +80,7 @@ func setupCacheableNodeExecutionContext(dataStore *storage.DataStore, taskTempla }, ) - var taskReader handler.TaskReader + var taskReader interfaces.TaskReader if taskTemplate != nil { taskReader = mockTaskReader{ taskTemplate: taskTemplate, @@ -99,33 +97,6 @@ func setupCacheableNodeExecutionContext(dataStore *storage.DataStore, taskTempla } } -func setupCacheableNodeExecutor(t *testing.T, catalogClient catalog.Client, dataStore *storage.DataStore, testScope promutils.Scope) *nodeExecutor { - ctx := context.TODO() - - adminClient := launchplan.NewFailFastLaunchPlanExecutor() - enqueueWorkflow := func(workflowID v1alpha1.WorkflowID) {} - eventConfig := &config.EventConfig{ - RawOutputPolicy: config.RawOutputPolicyReference, - } - fakeKubeClient := executorsmocks.NewFakeKubeClient() - maxDatasetSize := int64(10) - mockEventSink := eventsmocks.NewMockEventSink() - nodeConfig := config.GetConfig().NodeConfig - rawOutputPrefix := storage.DataReference("s3://bucket/") - recoveryClient := &recoverymocks.Client{} - testClusterID := "cluster1" - - nodeExecutorInterface, err := NewExecutor(ctx, nodeConfig, dataStore, enqueueWorkflow, mockEventSink, - adminClient, adminClient, maxDatasetSize, rawOutputPrefix, fakeKubeClient, catalogClient, - recoveryClient, eventConfig, testClusterID, signalClient, testScope.NewSubScope("node_executor")) - assert.NoError(t, err) - - nodeExecutor, ok := nodeExecutorInterface.(*nodeExecutor) - assert.True(t, ok) - - return nodeExecutor -} - func TestComputeCatalogReservationOwnerID(t *testing.T) { nCtx := setupCacheableNodeExecutionContext(nil, nil) @@ -240,8 +211,20 @@ func TestCheckCatalogCache(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { testScope := promutils.NewTestScope() + metrics := &nodeMetrics{ + catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", testScope), + catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", testScope), + catalogSkipCount: labeled.NewCounter("discovery_skip_count", "Task cached skipped in Discovery", testScope), + catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", testScope), + catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", testScope), + catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", testScope), + reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", testScope), + reservationGetSuccessCount: labeled.NewCounter("reservation_get_success_count", "Reservation GetOrExtend success count", testScope), + reservationReleaseFailureCount: labeled.NewCounter("reservation_release_failure_count", "Reservation Release failure count", testScope), + reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", testScope), + } - cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler := &interfacesmocks.CacheableNodeHandler{} cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(test.catalogKey, nil) catalogClient := &catalogmocks.Client{} @@ -255,7 +238,10 @@ func TestCheckCatalogCache(t *testing.T) { ) assert.NoError(t, err) - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nodeExecutor := &nodeExecutor{ + catalog: catalogClient, + metrics: metrics, + } nCtx := setupCacheableNodeExecutionContext(dataStore, nil) if test.preWriteOutputFile { @@ -304,8 +290,20 @@ func TestGetOrExtendCatalogReservation(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { testScope := promutils.NewTestScope() + metrics := &nodeMetrics{ + catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", testScope), + catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", testScope), + catalogSkipCount: labeled.NewCounter("discovery_skip_count", "Task cached skipped in Discovery", testScope), + catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", testScope), + catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", testScope), + catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", testScope), + reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", testScope), + reservationGetSuccessCount: labeled.NewCounter("reservation_get_success_count", "Reservation GetOrExtend success count", testScope), + reservationReleaseFailureCount: labeled.NewCounter("reservation_release_failure_count", "Reservation Release failure count", testScope), + reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", testScope), + } - cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler := &interfacesmocks.CacheableNodeHandler{} cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) catalogClient := &catalogmocks.Client{} @@ -316,7 +314,10 @@ func TestGetOrExtendCatalogReservation(t *testing.T) { nil, ) - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, nil, testScope) + nodeExecutor := &nodeExecutor{ + catalog: catalogClient, + metrics: metrics, + } nCtx := setupCacheableNodeExecutionContext(nil, &core.TaskTemplate{}) // execute catalog cache check @@ -350,14 +351,29 @@ func TestReleaseCatalogReservation(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { testScope := promutils.NewTestScope() + metrics := &nodeMetrics{ + catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", testScope), + catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", testScope), + catalogSkipCount: labeled.NewCounter("discovery_skip_count", "Task cached skipped in Discovery", testScope), + catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", testScope), + catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", testScope), + catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", testScope), + reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", testScope), + reservationGetSuccessCount: labeled.NewCounter("reservation_get_success_count", "Reservation GetOrExtend success count", testScope), + reservationReleaseFailureCount: labeled.NewCounter("reservation_release_failure_count", "Reservation Release failure count", testScope), + reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", testScope), + } - cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler := &interfacesmocks.CacheableNodeHandler{} cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(catalog.Key{}, nil) catalogClient := &catalogmocks.Client{} catalogClient.OnReleaseReservationMatch(mock.Anything, mock.Anything, mock.Anything).Return(test.releaseError) - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, nil, testScope) + nodeExecutor := &nodeExecutor{ + catalog: catalogClient, + metrics: metrics, + } nCtx := setupCacheableNodeExecutionContext(nil, &core.TaskTemplate{}) // execute catalog cache check @@ -409,8 +425,20 @@ func TestWriteCatalogCache(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { testScope := promutils.NewTestScope() + metrics := &nodeMetrics{ + catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", testScope), + catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", testScope), + catalogSkipCount: labeled.NewCounter("discovery_skip_count", "Task cached skipped in Discovery", testScope), + catalogPutSuccessCount: labeled.NewCounter("discovery_put_success_count", "Discovery Put success count", testScope), + catalogPutFailureCount: labeled.NewCounter("discovery_put_failure_count", "Discovery Put failure count", testScope), + catalogGetFailureCount: labeled.NewCounter("discovery_get_failure_count", "Discovery Get faillure count", testScope), + reservationGetFailureCount: labeled.NewCounter("reservation_get_failure_count", "Reservation GetOrExtend failure count", testScope), + reservationGetSuccessCount: labeled.NewCounter("reservation_get_success_count", "Reservation GetOrExtend success count", testScope), + reservationReleaseFailureCount: labeled.NewCounter("reservation_release_failure_count", "Reservation Release failure count", testScope), + reservationReleaseSuccessCount: labeled.NewCounter("reservation_release_success_count", "Reservation Release success count", testScope), + } - cacheableHandler := &handlermocks.CacheableNode{} + cacheableHandler := &interfacesmocks.CacheableNodeHandler{} cacheableHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything).Return(test.catalogKey, nil) catalogClient := &catalogmocks.Client{} @@ -424,7 +452,10 @@ func TestWriteCatalogCache(t *testing.T) { ) assert.NoError(t, err) - nodeExecutor := setupCacheableNodeExecutor(t, catalogClient, dataStore, testScope) + nodeExecutor := &nodeExecutor{ + catalog: catalogClient, + metrics: metrics, + } nCtx := setupCacheableNodeExecutionContext(dataStore, &core.TaskTemplate{}) // execute catalog cache check diff --git a/pkg/controller/nodes/dynamic/handler_test.go b/pkg/controller/nodes/dynamic/handler_test.go index b4e33b149..980e7601a 100644 --- a/pkg/controller/nodes/dynamic/handler_test.go +++ b/pkg/controller/nodes/dynamic/handler_test.go @@ -525,12 +525,11 @@ func Test_dynamicNodeHandler_Handle_SubTaskV1(t *testing.T) { } type args struct { - s interfaces.NodeStatus - isErr bool - dj *core.DynamicJobSpec - validErr *io.ExecutionError - validCacheStatus *catalog.Status - generateOutputs bool + s interfaces.NodeStatus + isErr bool + dj *core.DynamicJobSpec + validErr *io.ExecutionError + generateOutputs bool } type want struct { p handler.EPhase @@ -545,7 +544,7 @@ func Test_dynamicNodeHandler_Handle_SubTaskV1(t *testing.T) { }{ {"error", args{isErr: true, dj: createDynamicJobSpec()}, want{isErr: true}}, {"success", args{s: interfaces.NodeStatusSuccess, dj: createDynamicJobSpec()}, want{p: handler.EPhaseDynamicRunning, phase: v1alpha1.DynamicNodePhaseExecuting}}, - {"complete", args{s: interfaces.NodeStatusComplete, dj: createDynamicJobSpec(), generateOutputs: true, validCacheStatus: &validCachePopulatedStatus}, want{p: handler.EPhaseSuccess, phase: v1alpha1.DynamicNodePhaseExecuting, info: execInfoWithTaskNodeMeta}}, + {"complete", args{s: interfaces.NodeStatusComplete, dj: createDynamicJobSpec(), generateOutputs: true}, want{p: handler.EPhaseSuccess, phase: v1alpha1.DynamicNodePhaseExecuting, info: execInfoOutputOnly}}, {"complete-no-outputs", args{s: interfaces.NodeStatusComplete, dj: createDynamicJobSpec(), generateOutputs: false}, want{p: handler.EPhaseRetryableFailure, phase: v1alpha1.DynamicNodePhaseFailing}}, {"complete-valid-error-retryable", args{s: interfaces.NodeStatusComplete, dj: createDynamicJobSpec(), validErr: &io.ExecutionError{IsRecoverable: true}, generateOutputs: true}, want{p: handler.EPhaseRetryableFailure, phase: v1alpha1.DynamicNodePhaseFailing, info: execInfoOutputOnly}}, {"complete-valid-error", args{s: interfaces.NodeStatusComplete, dj: createDynamicJobSpec(), validErr: &io.ExecutionError{}, generateOutputs: true}, want{p: handler.EPhaseFailed, phase: v1alpha1.DynamicNodePhaseFailing, info: execInfoOutputOnly}}, diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index e51cb7fc8..4ac2a6632 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -1433,26 +1433,26 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora nodeScope := scope.NewSubScope("node") metrics := &nodeMetrics{ - Scope: nodeScope, - FailureDuration: labeled.NewStopWatch("failure_duration", "Indicates the total execution time of a failed workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - SuccessDuration: labeled.NewStopWatch("success_duration", "Indicates the total execution time of a successful workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - RecoveryDuration: labeled.NewStopWatch("recovery_duration", "Indicates the total execution time of a recovered workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - UserErrorDuration: labeled.NewStopWatch("user_error_duration", "Indicates the total execution time before user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - SystemErrorDuration: labeled.NewStopWatch("system_error_duration", "Indicates the total execution time before system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - UnknownErrorDuration: labeled.NewStopWatch("unknown_error_duration", "Indicates the total execution time before unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - PermanentUserErrorDuration: labeled.NewStopWatch("perma_user_error_duration", "Indicates the total execution time before non recoverable user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - PermanentSystemErrorDuration: labeled.NewStopWatch("perma_system_error_duration", "Indicates the total execution time before non recoverable system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - PermanentUnknownErrorDuration: labeled.NewStopWatch("perma_unknown_error_duration", "Indicates the total execution time before non recoverable unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - InputsWriteFailure: labeled.NewCounter("inputs_write_fail", "Indicates failure in writing node inputs to metastore", nodeScope), - TimedOutFailure: labeled.NewCounter("timeout_fail", "Indicates failure due to timeout", nodeScope), - InterruptedThresholdHit: labeled.NewCounter("interrupted_threshold", "Indicates the node interruptible disabled because it hit max failure count", nodeScope), - InterruptibleNodesRunning: labeled.NewCounter("interruptible_nodes_running", "number of interruptible nodes running", nodeScope), - InterruptibleNodesTerminated: labeled.NewCounter("interruptible_nodes_terminated", "number of interruptible nodes finished running", nodeScope), - ResolutionFailure: labeled.NewCounter("input_resolve_fail", "Indicates failure in resolving node inputs", nodeScope), - TransitionLatency: labeled.NewStopWatch("transition_latency", "Measures the latency between the last parent node stoppedAt time and current node's queued time.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - QueuingLatency: labeled.NewStopWatch("queueing_latency", "Measures the latency between the time a node's been queued to the time the handler reported the executable moved to running state", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), - NodeExecutionTime: labeled.NewStopWatch("node_exec_latency", "Measures the time taken to execute one node, a node can be complex so it may encompass sub-node latency.", time.Microsecond, nodeScope, labeled.EmitUnlabeledMetric), - NodeInputGatherLatency: labeled.NewStopWatch("node_input_latency", "Measures the latency to aggregate inputs and check readiness of a node", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + Scope: nodeScope, + FailureDuration: labeled.NewStopWatch("failure_duration", "Indicates the total execution time of a failed workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + SuccessDuration: labeled.NewStopWatch("success_duration", "Indicates the total execution time of a successful workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + RecoveryDuration: labeled.NewStopWatch("recovery_duration", "Indicates the total execution time of a recovered workflow.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + UserErrorDuration: labeled.NewStopWatch("user_error_duration", "Indicates the total execution time before user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + SystemErrorDuration: labeled.NewStopWatch("system_error_duration", "Indicates the total execution time before system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + UnknownErrorDuration: labeled.NewStopWatch("unknown_error_duration", "Indicates the total execution time before unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + PermanentUserErrorDuration: labeled.NewStopWatch("perma_user_error_duration", "Indicates the total execution time before non recoverable user error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + PermanentSystemErrorDuration: labeled.NewStopWatch("perma_system_error_duration", "Indicates the total execution time before non recoverable system error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + PermanentUnknownErrorDuration: labeled.NewStopWatch("perma_unknown_error_duration", "Indicates the total execution time before non recoverable unknown error", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + InputsWriteFailure: labeled.NewCounter("inputs_write_fail", "Indicates failure in writing node inputs to metastore", nodeScope), + TimedOutFailure: labeled.NewCounter("timeout_fail", "Indicates failure due to timeout", nodeScope), + InterruptedThresholdHit: labeled.NewCounter("interrupted_threshold", "Indicates the node interruptible disabled because it hit max failure count", nodeScope), + InterruptibleNodesRunning: labeled.NewCounter("interruptible_nodes_running", "number of interruptible nodes running", nodeScope), + InterruptibleNodesTerminated: labeled.NewCounter("interruptible_nodes_terminated", "number of interruptible nodes finished running", nodeScope), + ResolutionFailure: labeled.NewCounter("input_resolve_fail", "Indicates failure in resolving node inputs", nodeScope), + TransitionLatency: labeled.NewStopWatch("transition_latency", "Measures the latency between the last parent node stoppedAt time and current node's queued time.", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + QueuingLatency: labeled.NewStopWatch("queueing_latency", "Measures the latency between the time a node's been queued to the time the handler reported the executable moved to running state", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), + NodeExecutionTime: labeled.NewStopWatch("node_exec_latency", "Measures the time taken to execute one node, a node can be complex so it may encompass sub-node latency.", time.Microsecond, nodeScope, labeled.EmitUnlabeledMetric), + NodeInputGatherLatency: labeled.NewStopWatch("node_input_latency", "Measures the latency to aggregate inputs and check readiness of a node", time.Millisecond, nodeScope, labeled.EmitUnlabeledMetric), catalogHitCount: labeled.NewCounter("discovery_hit_count", "Task cached in Discovery", scope), catalogMissCount: labeled.NewCounter("discovery_miss_count", "Task not cached in Discovery", scope), catalogSkipCount: labeled.NewCounter("discovery_skip_count", "Task cached skipped in Discovery", scope), diff --git a/pkg/controller/nodes/executor_test.go b/pkg/controller/nodes/executor_test.go index 74efd0ed1..3a474b39e 100644 --- a/pkg/controller/nodes/executor_test.go +++ b/pkg/controller/nodes/executor_test.go @@ -2607,7 +2607,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { } } - setupNodeExecutor := func(t *testing.T, catalogClient pluginscatalog.Client, dataStore *storage.DataStore, mockHandler handler.CacheableNode, testScope promutils.Scope) *nodeExecutor { + setupNodeExecutor := func(t *testing.T, catalogClient pluginscatalog.Client, dataStore *storage.DataStore, mockHandler interfaces.CacheableNodeHandler, testScope promutils.Scope) interfaces.Node { ctx := context.TODO() // create mocks @@ -2625,19 +2625,13 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { testClusterID := "cluster1" // initialize node executor - nodeExecutorInterface, err := NewExecutor(ctx, nodeConfig, dataStore, enqueueWorkflow, mockEventSink, + mockHandlerFactory := &nodemocks.HandlerFactory{} + mockHandlerFactory.OnGetHandler(v1alpha1.NodeKindTask).Return(mockHandler, nil) + nodeExecutor, err := NewExecutor(ctx, nodeConfig, dataStore, enqueueWorkflow, mockEventSink, adminClient, adminClient, maxDatasetSize, rawOutputPrefix, fakeKubeClient, catalogClient, - recoveryClient, eventConfig, testClusterID, signalClient, testScope) + recoveryClient, eventConfig, testClusterID, signalClient, mockHandlerFactory, testScope) assert.NoError(t, err) - nodeExecutor, ok := nodeExecutorInterface.(*nodeExecutor) - assert.True(t, ok) - - // setup node handler - mockHandlerFactory := &mocks2.HandlerFactory{} - mockHandlerFactory.OnGetHandler(v1alpha1.NodeKindTask).Return(mockHandler, nil) - nodeExecutor.nodeHandlerFactory = mockHandlerFactory - return nodeExecutor } @@ -2743,14 +2737,14 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { catalogClient.OnGetOrExtendReservationMatch(mock.Anything, mock.Anything, mock.Anything, mock.Anything). Return(&datacatalog.Reservation{OwnerId: test.cacheReservationOwnerID}, nil) - mockHandler := &nodeHandlerMocks.CacheableNode{} + mockHandler := &nodemocks.CacheableNodeHandler{} mockHandler.OnIsCacheableMatch( mock.Anything, - mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == currentNodeID }), + mock.MatchedBy(func(nCtx interfaces.NodeExecutionContext) bool { return nCtx.NodeID() == currentNodeID }), ).Return(test.cacheable, test.cacheSerializable, nil) mockHandler.OnIsCacheableMatch( mock.Anything, - mock.MatchedBy(func(nCtx handler.NodeExecutionContext) bool { return nCtx.NodeID() == downstreamNodeID }), + mock.MatchedBy(func(nCtx interfaces.NodeExecutionContext) bool { return nCtx.NodeID() == downstreamNodeID }), ).Return(false, false, nil) mockHandler.OnGetCatalogKeyMatch(mock.Anything, mock.Anything). Return(pluginscatalog.Key{Identifier: core.Identifier{Name: currentNodeID}}, nil) diff --git a/pkg/controller/nodes/interfaces/mocks/cacheable_node.go b/pkg/controller/nodes/interfaces/mocks/cacheable_node.go deleted file mode 100644 index 5e5ba50a7..000000000 --- a/pkg/controller/nodes/interfaces/mocks/cacheable_node.go +++ /dev/null @@ -1,270 +0,0 @@ -// Code generated by mockery v1.0.1. DO NOT EDIT. - -package mocks - -import ( - context "context" - - catalog "github.com/flyteorg/flyteplugins/go/tasks/pluginmachinery/catalog" - - handler "github.com/flyteorg/flytepropeller/pkg/controller/nodes/handler" - - mock "github.com/stretchr/testify/mock" -) - -// CacheableNode is an autogenerated mock type for the CacheableNode type -type CacheableNode struct { - mock.Mock -} - -type CacheableNode_Abort struct { - *mock.Call -} - -func (_m CacheableNode_Abort) Return(_a0 error) *CacheableNode_Abort { - return &CacheableNode_Abort{Call: _m.Call.Return(_a0)} -} - -func (_m *CacheableNode) OnAbort(ctx context.Context, executionContext handler.NodeExecutionContext, reason string) *CacheableNode_Abort { - c_call := _m.On("Abort", ctx, executionContext, reason) - return &CacheableNode_Abort{Call: c_call} -} - -func (_m *CacheableNode) OnAbortMatch(matchers ...interface{}) *CacheableNode_Abort { - c_call := _m.On("Abort", matchers...) - return &CacheableNode_Abort{Call: c_call} -} - -// Abort provides a mock function with given fields: ctx, executionContext, reason -func (_m *CacheableNode) Abort(ctx context.Context, executionContext handler.NodeExecutionContext, reason string) error { - ret := _m.Called(ctx, executionContext, reason) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext, string) error); ok { - r0 = rf(ctx, executionContext, reason) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -type CacheableNode_Finalize struct { - *mock.Call -} - -func (_m CacheableNode_Finalize) Return(_a0 error) *CacheableNode_Finalize { - return &CacheableNode_Finalize{Call: _m.Call.Return(_a0)} -} - -func (_m *CacheableNode) OnFinalize(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_Finalize { - c_call := _m.On("Finalize", ctx, executionContext) - return &CacheableNode_Finalize{Call: c_call} -} - -func (_m *CacheableNode) OnFinalizeMatch(matchers ...interface{}) *CacheableNode_Finalize { - c_call := _m.On("Finalize", matchers...) - return &CacheableNode_Finalize{Call: c_call} -} - -// Finalize provides a mock function with given fields: ctx, executionContext -func (_m *CacheableNode) Finalize(ctx context.Context, executionContext handler.NodeExecutionContext) error { - ret := _m.Called(ctx, executionContext) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) error); ok { - r0 = rf(ctx, executionContext) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -type CacheableNode_FinalizeRequired struct { - *mock.Call -} - -func (_m CacheableNode_FinalizeRequired) Return(_a0 bool) *CacheableNode_FinalizeRequired { - return &CacheableNode_FinalizeRequired{Call: _m.Call.Return(_a0)} -} - -func (_m *CacheableNode) OnFinalizeRequired() *CacheableNode_FinalizeRequired { - c_call := _m.On("FinalizeRequired") - return &CacheableNode_FinalizeRequired{Call: c_call} -} - -func (_m *CacheableNode) OnFinalizeRequiredMatch(matchers ...interface{}) *CacheableNode_FinalizeRequired { - c_call := _m.On("FinalizeRequired", matchers...) - return &CacheableNode_FinalizeRequired{Call: c_call} -} - -// FinalizeRequired provides a mock function with given fields: -func (_m *CacheableNode) FinalizeRequired() bool { - ret := _m.Called() - - var r0 bool - if rf, ok := ret.Get(0).(func() bool); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(bool) - } - - return r0 -} - -type CacheableNode_GetCatalogKey struct { - *mock.Call -} - -func (_m CacheableNode_GetCatalogKey) Return(_a0 catalog.Key, _a1 error) *CacheableNode_GetCatalogKey { - return &CacheableNode_GetCatalogKey{Call: _m.Call.Return(_a0, _a1)} -} - -func (_m *CacheableNode) OnGetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_GetCatalogKey { - c_call := _m.On("GetCatalogKey", ctx, executionContext) - return &CacheableNode_GetCatalogKey{Call: c_call} -} - -func (_m *CacheableNode) OnGetCatalogKeyMatch(matchers ...interface{}) *CacheableNode_GetCatalogKey { - c_call := _m.On("GetCatalogKey", matchers...) - return &CacheableNode_GetCatalogKey{Call: c_call} -} - -// GetCatalogKey provides a mock function with given fields: ctx, executionContext -func (_m *CacheableNode) GetCatalogKey(ctx context.Context, executionContext handler.NodeExecutionContext) (catalog.Key, error) { - ret := _m.Called(ctx, executionContext) - - var r0 catalog.Key - if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) catalog.Key); ok { - r0 = rf(ctx, executionContext) - } else { - r0 = ret.Get(0).(catalog.Key) - } - - var r1 error - if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) error); ok { - r1 = rf(ctx, executionContext) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -type CacheableNode_Handle struct { - *mock.Call -} - -func (_m CacheableNode_Handle) Return(_a0 handler.Transition, _a1 error) *CacheableNode_Handle { - return &CacheableNode_Handle{Call: _m.Call.Return(_a0, _a1)} -} - -func (_m *CacheableNode) OnHandle(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_Handle { - c_call := _m.On("Handle", ctx, executionContext) - return &CacheableNode_Handle{Call: c_call} -} - -func (_m *CacheableNode) OnHandleMatch(matchers ...interface{}) *CacheableNode_Handle { - c_call := _m.On("Handle", matchers...) - return &CacheableNode_Handle{Call: c_call} -} - -// Handle provides a mock function with given fields: ctx, executionContext -func (_m *CacheableNode) Handle(ctx context.Context, executionContext handler.NodeExecutionContext) (handler.Transition, error) { - ret := _m.Called(ctx, executionContext) - - var r0 handler.Transition - if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) handler.Transition); ok { - r0 = rf(ctx, executionContext) - } else { - r0 = ret.Get(0).(handler.Transition) - } - - var r1 error - if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) error); ok { - r1 = rf(ctx, executionContext) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -type CacheableNode_IsCacheable struct { - *mock.Call -} - -func (_m CacheableNode_IsCacheable) Return(_a0 bool, _a1 bool, _a2 error) *CacheableNode_IsCacheable { - return &CacheableNode_IsCacheable{Call: _m.Call.Return(_a0, _a1, _a2)} -} - -func (_m *CacheableNode) OnIsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) *CacheableNode_IsCacheable { - c_call := _m.On("IsCacheable", ctx, executionContext) - return &CacheableNode_IsCacheable{Call: c_call} -} - -func (_m *CacheableNode) OnIsCacheableMatch(matchers ...interface{}) *CacheableNode_IsCacheable { - c_call := _m.On("IsCacheable", matchers...) - return &CacheableNode_IsCacheable{Call: c_call} -} - -// IsCacheable provides a mock function with given fields: ctx, executionContext -func (_m *CacheableNode) IsCacheable(ctx context.Context, executionContext handler.NodeExecutionContext) (bool, bool, error) { - ret := _m.Called(ctx, executionContext) - - var r0 bool - if rf, ok := ret.Get(0).(func(context.Context, handler.NodeExecutionContext) bool); ok { - r0 = rf(ctx, executionContext) - } else { - r0 = ret.Get(0).(bool) - } - - var r1 bool - if rf, ok := ret.Get(1).(func(context.Context, handler.NodeExecutionContext) bool); ok { - r1 = rf(ctx, executionContext) - } else { - r1 = ret.Get(1).(bool) - } - - var r2 error - if rf, ok := ret.Get(2).(func(context.Context, handler.NodeExecutionContext) error); ok { - r2 = rf(ctx, executionContext) - } else { - r2 = ret.Error(2) - } - - return r0, r1, r2 -} - -type CacheableNode_Setup struct { - *mock.Call -} - -func (_m CacheableNode_Setup) Return(_a0 error) *CacheableNode_Setup { - return &CacheableNode_Setup{Call: _m.Call.Return(_a0)} -} - -func (_m *CacheableNode) OnSetup(ctx context.Context, setupContext handler.SetupContext) *CacheableNode_Setup { - c_call := _m.On("Setup", ctx, setupContext) - return &CacheableNode_Setup{Call: c_call} -} - -func (_m *CacheableNode) OnSetupMatch(matchers ...interface{}) *CacheableNode_Setup { - c_call := _m.On("Setup", matchers...) - return &CacheableNode_Setup{Call: c_call} -} - -// Setup provides a mock function with given fields: ctx, setupContext -func (_m *CacheableNode) Setup(ctx context.Context, setupContext handler.SetupContext) error { - ret := _m.Called(ctx, setupContext) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, handler.SetupContext) error); ok { - r0 = rf(ctx, setupContext) - } else { - r0 = ret.Error(0) - } - - return r0 -} From 95bf8e16efeaf7b4995fd69961304de37ba23676 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 3 Aug 2023 16:48:36 -0500 Subject: [PATCH 32/40] removed unnecessary async catalog client from nodeExecutor Signed-off-by: Daniel Rammer --- pkg/controller/nodes/executor.go | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 4ac2a6632..47a0c5d07 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -477,7 +477,6 @@ func (c *recursiveNodeExecutor) WithNodeExecutionContextBuilder(nCtxBuilder inte // nodeExecutor implements the NodeExecutor interface and is responsible for executing a single node. type nodeExecutor struct { - asyncCatalog catalog.AsyncClient catalog catalog.Client clusterID string defaultActiveDeadline time.Duration @@ -1422,15 +1421,6 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora return nil, err } - async, err := catalog.NewAsyncClient(catalogClient, *catalog.GetConfig(), scope.NewSubScope("async_catalog")) - if err != nil { - return nil, err - } - - if err = async.Start(ctx); err != nil { - return nil, err - } - nodeScope := scope.NewSubScope("node") metrics := &nodeMetrics{ Scope: nodeScope, @@ -1466,7 +1456,6 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora } nodeExecutor := &nodeExecutor{ - asyncCatalog: async, catalog: catalogClient, clusterID: clusterID, defaultActiveDeadline: nodeConfig.DefaultDeadlines.DefaultNodeActiveDeadline.Duration, From 5d019bf47f05d59f2ac2b2f85f34e89b8c0bb65d Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Fri, 4 Aug 2023 13:07:02 -0500 Subject: [PATCH 33/40] general refactoring Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache.go | 5 +++-- pkg/controller/nodes/interfaces/node_exec_context.go | 1 - 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 9ba3081ea..dc762877e 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -81,7 +81,8 @@ func updatePhaseCacheInfo(phaseInfo handler.PhaseInfo, cacheStatus *catalog.Stat // CheckCatalogCache uses the handler and contexts to check if cached outputs for the current node // exist. If the exist, this function also copies the outputs to this node. func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx interfaces.NodeExecutionContext, cacheHandler interfaces.CacheableNodeHandler) (catalog.Entry, error) { - // Check if the outputs file already exists. Since processing downstream nodes is performed + // TODO @hamersaw - we don't need this anymore since we're not auto handleDownstream? + /*// Check if the outputs file already exists. Since processing downstream nodes is performed // immediately following a cache hit, any failures will result in performing this operation // again during the following node evaluation. In the scenario that outputs were already // written we report a cache hit to ensure correctness in node evaluations. @@ -94,7 +95,7 @@ func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx interfaces.No if metadata.Exists() { return catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil)), nil - } + }*/ catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { diff --git a/pkg/controller/nodes/interfaces/node_exec_context.go b/pkg/controller/nodes/interfaces/node_exec_context.go index 3b8afc384..fa9d32388 100644 --- a/pkg/controller/nodes/interfaces/node_exec_context.go +++ b/pkg/controller/nodes/interfaces/node_exec_context.go @@ -54,7 +54,6 @@ type NodeExecutionContext interface { DataStore() *storage.DataStore InputReader() io.InputReader - //EventsRecorder() events.TaskEventRecorder EventsRecorder() EventRecorder NodeID() v1alpha1.NodeID Node() v1alpha1.ExecutableNode From e0f671563aa1dcabdaa0bf37e8cf99efdca48288 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 7 Aug 2023 09:01:38 -0500 Subject: [PATCH 34/40] fastcache working with arraynode Signed-off-by: Daniel Rammer --- pkg/controller/nodes/array/handler.go | 10 ++----- pkg/controller/nodes/array/node_lookup.go | 8 ++++++ pkg/controller/nodes/cache.go | 1 + pkg/controller/nodes/executor.go | 34 ----------------------- 4 files changed, 12 insertions(+), 41 deletions(-) diff --git a/pkg/controller/nodes/array/handler.go b/pkg/controller/nodes/array/handler.go index 19641cb93..ae559a2b9 100644 --- a/pkg/controller/nodes/array/handler.go +++ b/pkg/controller/nodes/array/handler.go @@ -543,19 +543,15 @@ func (a *arrayNodeHandler) buildArrayNodeContext(ctx context.Context, nCtx inter inputReader := newStaticInputReader(nCtx.InputReader(), inputLiteralMap) - // if node has not yet started we automatically set to NodePhaseQueued to skip input resolution - if nodePhase == v1alpha1.NodePhaseNotYetStarted { - // TODO - to supprt fastcache we'll need to override the bindings to BindingScalars for the input resolution on the nCtx - // that way resolution is just reading a literal ... but does this still write a file then?!? - nodePhase = v1alpha1.NodePhaseQueued - } - // wrap node lookup subNodeSpec := *arrayNode.GetSubNodeSpec() subNodeID := fmt.Sprintf("%s-n%d", nCtx.NodeID(), subNodeIndex) subNodeSpec.ID = subNodeID subNodeSpec.Name = subNodeID + subNodeSpec.InputBindings = nil // TODO @hamersaw - mock input bindings to nil to bypass input resolution + // alternatively we could mock the input bindings to BindingScalars to bypass input resolution + // and mock the datastore to bypass writign input resolution - this seems easier // TODO - if we want to support more plugin types we need to figure out the best way to store plugin state // currently just mocking based on node phase -> which works for all k8s plugins diff --git a/pkg/controller/nodes/array/node_lookup.go b/pkg/controller/nodes/array/node_lookup.go index 061b323af..b3b8dd03a 100644 --- a/pkg/controller/nodes/array/node_lookup.go +++ b/pkg/controller/nodes/array/node_lookup.go @@ -14,6 +14,14 @@ type arrayNodeLookup struct { subNodeStatus *v1alpha1.NodeStatus } +func (a *arrayNodeLookup) ToNode(id v1alpha1.NodeID) ([]v1alpha1.NodeID, error) { + if id == a.subNodeID { + return nil, nil + } + + return a.NodeLookup.ToNode(id) +} + func (a *arrayNodeLookup) GetNode(nodeID v1alpha1.NodeID) (v1alpha1.ExecutableNode, bool) { if nodeID == a.subNodeID { return a.subNodeSpec, true diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index dc762877e..9634f7ce2 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -138,6 +138,7 @@ func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx interfaces.No return catalog.Entry{}, nodeserrors.Errorf(nodeserrors.IllegalStateError, nCtx.NodeID(), "execution error from a cache output, bad state: %s", ee.String()) } + outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) if err := nCtx.DataStore().WriteProtobuf(ctx, outputFile, storage.Options{}, o); err != nil { logger.Errorf(ctx, "failed to write cached value to datastore, err: %s", err.Error()) return catalog.Entry{}, err diff --git a/pkg/controller/nodes/executor.go b/pkg/controller/nodes/executor.go index 47a0c5d07..011acd0bb 100644 --- a/pkg/controller/nodes/executor.go +++ b/pkg/controller/nodes/executor.go @@ -512,40 +512,6 @@ func (c *nodeExecutor) RecordTransitionLatency(ctx context.Context, dag executor } } -/*func (c *nodeExecutor) IdempotentRecordEvent(ctx context.Context, nodeEvent *event.NodeExecutionEvent) error { - if nodeEvent == nil { - return fmt.Errorf("event recording attempt of Nil Node execution event") - } - - if nodeEvent.Id == nil { - return fmt.Errorf("event recording attempt of with nil node Event ID") - } - - logger.Infof(ctx, "Recording NodeEvent [%s] phase[%s]", nodeEvent.GetId().String(), nodeEvent.Phase.String()) - err := c.nodeRecorder.RecordNodeEvent(ctx, nodeEvent, c.eventConfig) - if err != nil { - if nodeEvent.GetId().NodeId == v1alpha1.EndNodeID { - return nil - } - - if eventsErr.IsAlreadyExists(err) { - logger.Infof(ctx, "Node event phase: %s, nodeId %s already exist", - nodeEvent.Phase.String(), nodeEvent.GetId().NodeId) - return nil - } else if eventsErr.IsEventAlreadyInTerminalStateError(err) { - if IsTerminalNodePhase(nodeEvent.Phase) { - // Event was trying to record a different terminal phase for an already terminal event. ignoring. - logger.Infof(ctx, "Node event phase: %s, nodeId %s already in terminal phase. err: %s", - nodeEvent.Phase.String(), nodeEvent.GetId().NodeId, err.Error()) - return nil - } - logger.Warningf(ctx, "Failed to record nodeEvent, error [%s]", err.Error()) - return errors.Wrapf(errors.IllegalStateError, nodeEvent.Id.NodeId, err, "phase mis-match mismatch between propeller and control plane; Trying to record Node p: %s", nodeEvent.Phase) - } - } - return err -}*/ - func (c *nodeExecutor) recoverInputs(ctx context.Context, nCtx interfaces.NodeExecutionContext, recovered *admin.NodeExecution, recoveredData *admin.NodeExecutionGetDataResponse) (*core.LiteralMap, error) { From f3ca83c512175975ed610122ac610af6d2c98740 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 7 Aug 2023 09:13:00 -0500 Subject: [PATCH 35/40] fixed unit tests - no longer checking for output existance on first execution of cached Signed-off-by: Daniel Rammer --- pkg/controller/nodes/cache_test.go | 21 --------------------- 1 file changed, 21 deletions(-) diff --git a/pkg/controller/nodes/cache_test.go b/pkg/controller/nodes/cache_test.go index ab77ff754..6d9c4fce9 100644 --- a/pkg/controller/nodes/cache_test.go +++ b/pkg/controller/nodes/cache_test.go @@ -148,7 +148,6 @@ func TestCheckCatalogCache(t *testing.T) { cacheError error catalogKey catalog.Key expectedCacheStatus core.CatalogCacheStatus - preWriteOutputFile bool assertOutputFile bool outputFileExists bool }{ @@ -160,7 +159,6 @@ func TestCheckCatalogCache(t *testing.T) { core.CatalogCacheStatus_CACHE_MISS, false, false, - false, }, { "CacheHitWithOutputs", @@ -179,7 +177,6 @@ func TestCheckCatalogCache(t *testing.T) { }, }, core.CatalogCacheStatus_CACHE_HIT, - false, true, true, }, @@ -192,20 +189,9 @@ func TestCheckCatalogCache(t *testing.T) { nil, catalog.Key{}, core.CatalogCacheStatus_CACHE_HIT, - false, true, false, }, - { - "OutputsAlreadyExist", - catalog.Entry{}, - nil, - catalog.Key{}, - core.CatalogCacheStatus_CACHE_HIT, - true, - true, - true, - }, } for _, test := range tests { @@ -244,13 +230,6 @@ func TestCheckCatalogCache(t *testing.T) { } nCtx := setupCacheableNodeExecutionContext(dataStore, nil) - if test.preWriteOutputFile { - // write mock data to outputs - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) - err = nCtx.DataStore().WriteProtobuf(context.TODO(), outputFile, storage.Options{}, &core.LiteralMap{}) - assert.NoError(t, err) - } - // execute catalog cache check cacheEntry, err := nodeExecutor.CheckCatalogCache(context.TODO(), nCtx, cacheableHandler) assert.NoError(t, err) From 15c46afca99d9c806e79073d94ed7fefbbe4ba97 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 7 Aug 2023 10:49:18 -0500 Subject: [PATCH 36/40] updating documentation TODOs Signed-off-by: Daniel Rammer --- pkg/controller/nodes/array/handler.go | 8 +++++--- pkg/controller/nodes/cache.go | 16 ---------------- 2 files changed, 5 insertions(+), 19 deletions(-) diff --git a/pkg/controller/nodes/array/handler.go b/pkg/controller/nodes/array/handler.go index ae559a2b9..02fc732a4 100644 --- a/pkg/controller/nodes/array/handler.go +++ b/pkg/controller/nodes/array/handler.go @@ -549,9 +549,11 @@ func (a *arrayNodeHandler) buildArrayNodeContext(ctx context.Context, nCtx inter subNodeID := fmt.Sprintf("%s-n%d", nCtx.NodeID(), subNodeIndex) subNodeSpec.ID = subNodeID subNodeSpec.Name = subNodeID - subNodeSpec.InputBindings = nil // TODO @hamersaw - mock input bindings to nil to bypass input resolution - // alternatively we could mock the input bindings to BindingScalars to bypass input resolution - // and mock the datastore to bypass writign input resolution - this seems easier + // mock the input bindings for the subNode to nil to bypass input resolution in the + // `nodeExecutor.preExecute` function. this is required because this function is the entrypoint + // for initial cache lookups. an alternative solution would be to mock the datastore to bypass + // writing the inputFile. + subNodeSpec.InputBindings = nil // TODO - if we want to support more plugin types we need to figure out the best way to store plugin state // currently just mocking based on node phase -> which works for all k8s plugins diff --git a/pkg/controller/nodes/cache.go b/pkg/controller/nodes/cache.go index 9634f7ce2..fbae02b16 100644 --- a/pkg/controller/nodes/cache.go +++ b/pkg/controller/nodes/cache.go @@ -81,22 +81,6 @@ func updatePhaseCacheInfo(phaseInfo handler.PhaseInfo, cacheStatus *catalog.Stat // CheckCatalogCache uses the handler and contexts to check if cached outputs for the current node // exist. If the exist, this function also copies the outputs to this node. func (n *nodeExecutor) CheckCatalogCache(ctx context.Context, nCtx interfaces.NodeExecutionContext, cacheHandler interfaces.CacheableNodeHandler) (catalog.Entry, error) { - // TODO @hamersaw - we don't need this anymore since we're not auto handleDownstream? - /*// Check if the outputs file already exists. Since processing downstream nodes is performed - // immediately following a cache hit, any failures will result in performing this operation - // again during the following node evaluation. In the scenario that outputs were already - // written we report a cache hit to ensure correctness in node evaluations. - outputFile := v1alpha1.GetOutputsFile(nCtx.NodeStatus().GetOutputDir()) - metadata, err := nCtx.DataStore().Head(ctx, outputFile) - if err != nil { - logger.Errorf(ctx, "Failed to check the existence of outputs file. err: '%v'", err) - return catalog.Entry{}, errors.Wrapf(err, "failed to check the existence of deck file.") - } - - if metadata.Exists() { - return catalog.NewCatalogEntry(nil, catalog.NewStatus(core.CatalogCacheStatus_CACHE_HIT, nil)), nil - }*/ - catalogKey, err := cacheHandler.GetCatalogKey(ctx, nCtx) if err != nil { return catalog.Entry{}, errors.Wrapf(err, "failed to initialize the catalogKey") From 9f8b7d925146a09a1026aed41a6d766ddf038bed Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 8 Aug 2023 13:07:07 -0500 Subject: [PATCH 37/40] updated arraynode fastcache to correctly report cache hits Signed-off-by: Daniel Rammer --- pkg/controller/nodes/array/handler.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/pkg/controller/nodes/array/handler.go b/pkg/controller/nodes/array/handler.go index 02fc732a4..396dc3f4c 100644 --- a/pkg/controller/nodes/array/handler.go +++ b/pkg/controller/nodes/array/handler.go @@ -295,6 +295,18 @@ func (a *arrayNodeHandler) Handle(ctx context.Context, nCtx interfaces.NodeExecu retryAttempt := subNodeStatus.GetAttempts() + // fastcache will not emit task events for cache hits. we need to manually detect a + // transition to `SUCCEEDED` and add an `ExternalResourceInfo` for it. + if cacheStatus == idlcore.CatalogCacheStatus_CACHE_HIT && len(arrayEventRecorder.TaskEvents()) == 0 { + externalResources = append(externalResources, &event.ExternalResourceInfo{ + ExternalId: buildSubNodeID(nCtx, i, retryAttempt), + Index: uint32(i), + RetryAttempt: retryAttempt, + Phase: idlcore.TaskExecution_SUCCEEDED, + CacheStatus: cacheStatus, + }) + } + for _, taskExecutionEvent := range arrayEventRecorder.TaskEvents() { for _, log := range taskExecutionEvent.Logs { log.Name = fmt.Sprintf("%s-%d", log.Name, i) @@ -476,6 +488,7 @@ func (a *arrayNodeHandler) Handle(ctx context.Context, nCtx interfaces.NodeExecu arrayNodeState.TaskPhaseVersion = taskPhaseVersion + 1 } + fmt.Printf("HAMERSAW - sending event with taskPhase %+v and version %d\n", taskPhase, taskPhaseVersion) taskExecutionEvent, err := buildTaskExecutionEvent(ctx, nCtx, taskPhase, taskPhaseVersion, externalResources) if err != nil { return handler.UnknownTransition, err From 4347b6d2c725809b119fedf07105467bc318b8c4 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Wed, 9 Aug 2023 10:29:26 -0500 Subject: [PATCH 38/40] remove print statement Signed-off-by: Daniel Rammer --- pkg/controller/nodes/array/handler.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/controller/nodes/array/handler.go b/pkg/controller/nodes/array/handler.go index 396dc3f4c..a8cc13c85 100644 --- a/pkg/controller/nodes/array/handler.go +++ b/pkg/controller/nodes/array/handler.go @@ -488,7 +488,6 @@ func (a *arrayNodeHandler) Handle(ctx context.Context, nCtx interfaces.NodeExecu arrayNodeState.TaskPhaseVersion = taskPhaseVersion + 1 } - fmt.Printf("HAMERSAW - sending event with taskPhase %+v and version %d\n", taskPhase, taskPhaseVersion) taskExecutionEvent, err := buildTaskExecutionEvent(ctx, nCtx, taskPhase, taskPhaseVersion, externalResources) if err != nil { return handler.UnknownTransition, err From 9245f3e4f9c9bad3db77bfef267a92a064c66f1d Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 14 Aug 2023 10:36:51 -0500 Subject: [PATCH 39/40] fixed cache serialize Signed-off-by: Daniel Rammer --- pkg/apis/flyteworkflow/v1alpha1/node_status.go | 2 +- pkg/controller/nodes/transformers.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/apis/flyteworkflow/v1alpha1/node_status.go b/pkg/apis/flyteworkflow/v1alpha1/node_status.go index 07787d6e7..b1f269989 100644 --- a/pkg/apis/flyteworkflow/v1alpha1/node_status.go +++ b/pkg/apis/flyteworkflow/v1alpha1/node_status.go @@ -595,7 +595,7 @@ func (in *NodeStatus) GetOrCreateArrayNodeStatus() MutableArrayNodeStatus { } func (in *NodeStatus) UpdatePhase(p NodePhase, occurredAt metav1.Time, reason string, err *core.ExecutionError) { - if in.Phase == p { + if in.Phase == p && in.Message == reason { // We will not update the phase multiple times. This prevents the comparison from returning false positive return } diff --git a/pkg/controller/nodes/transformers.go b/pkg/controller/nodes/transformers.go index b8c06ce3d..6e09c103b 100644 --- a/pkg/controller/nodes/transformers.go +++ b/pkg/controller/nodes/transformers.go @@ -229,8 +229,8 @@ func ToK8sTime(t time.Time) v1.Time { } func UpdateNodeStatus(np v1alpha1.NodePhase, p handler.PhaseInfo, n interfaces.NodeStateReader, s v1alpha1.ExecutableNodeStatus) { - // We update the phase only if it is not already updated - if np != s.GetPhase() { + // We update the phase and / or reason only if they are not already updated + if np != s.GetPhase() || p.GetReason() != s.GetMessage() { s.UpdatePhase(np, ToK8sTime(p.GetOccurredAt()), p.GetReason(), p.GetErr()) } // Update TaskStatus From c672b07c41d881a86cef60c68f87ba7c28aa8dbb Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Mon, 14 Aug 2023 14:42:56 -0500 Subject: [PATCH 40/40] fixed unit tests Signed-off-by: Daniel Rammer --- pkg/apis/flyteworkflow/v1alpha1/node_status_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/apis/flyteworkflow/v1alpha1/node_status_test.go b/pkg/apis/flyteworkflow/v1alpha1/node_status_test.go index 1fbf4391d..698b437eb 100644 --- a/pkg/apis/flyteworkflow/v1alpha1/node_status_test.go +++ b/pkg/apis/flyteworkflow/v1alpha1/node_status_test.go @@ -263,7 +263,8 @@ func TestNodeStatus_UpdatePhase(t *testing.T) { t.Run("identical-phase", func(t *testing.T) { p := NodePhaseQueued ns := NodeStatus{ - Phase: p, + Phase: p, + Message: queued, } msg := queued ns.UpdatePhase(p, n, msg, nil)