From 8437865efbaa9e35ce767ef92d9d36615896f4af Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Fri, 23 Feb 2024 13:01:31 -0800 Subject: [PATCH 01/25] Overlap create execution blob store reads/writes This change modifies launch paths stemming from `launchExecutionAndPrepareModel` to overlap blob store write and read calls, which dominate end-to-end latency (as seen in the traces below). Signed-off-by: Andrew Dye --- .../pkg/manager/impl/execution_manager.go | 121 +++++++++++++----- .../manager/impl/execution_manager_test.go | 10 ++ 2 files changed, 99 insertions(+), 32 deletions(-) diff --git a/flyteadmin/pkg/manager/impl/execution_manager.go b/flyteadmin/pkg/manager/impl/execution_manager.go index 6ae9a61a52..113c3b08a8 100644 --- a/flyteadmin/pkg/manager/impl/execution_manager.go +++ b/flyteadmin/pkg/manager/impl/execution_manager.go @@ -11,6 +11,7 @@ import ( "github.com/golang/protobuf/ptypes" "github.com/golang/protobuf/ptypes/timestamp" "github.com/prometheus/client_golang/prometheus" + "golang.org/x/sync/errgroup" "google.golang.org/grpc/codes" "github.com/flyteorg/flyte/flyteadmin/auth" @@ -445,7 +446,7 @@ func (m *ExecutionManager) launchSingleTaskExecution( return nil, nil, err } - // Prepare a skeleton workflow + // Prepare a skeleton workflow and launch plan taskIdentifier := request.Spec.LaunchPlan workflowModel, err := util.CreateOrGetWorkflowModel(ctx, request, m.db, m.workflowManager, m.namedEntityManager, taskIdentifier, &task) @@ -457,13 +458,7 @@ func (m *ExecutionManager) launchSingleTaskExecution( if err != nil { return nil, nil, err } - closure, err := util.FetchAndGetWorkflowClosure(ctx, m.storageClient, workflowModel.RemoteClosureIdentifier) - if err != nil { - return nil, nil, err - } - closure.CreatedAt = workflow.Closure.CreatedAt - workflow.Closure = closure - // Also prepare a skeleton launch plan. + launchPlan, err := util.CreateOrGetLaunchPlan(ctx, m.db, m.config, taskIdentifier, workflow.Closure.CompiledWorkflow.Primary.Template.Interface, workflowModel.ID, request.Spec) if err != nil { @@ -488,6 +483,40 @@ func (m *ExecutionManager) launchSingleTaskExecution( Domain: request.Domain, Name: name, } + + // Overlap the blob store reads and writes + getClosureGroup, getClosureGroupCtx := errgroup.WithContext(ctx) + var closure *admin.WorkflowClosure + getClosureGroup.Go(func() error { + var err error + closure, err = util.FetchAndGetWorkflowClosure(getClosureGroupCtx, m.storageClient, workflowModel.RemoteClosureIdentifier) + return err + }) + + offloadInputsGroup, offloadInputsGroupCtx := errgroup.WithContext(ctx) + var inputsURI storage.DataReference + offloadInputsGroup.Go(func() error { + var err error + inputsURI, err = common.OffloadLiteralMap(offloadInputsGroupCtx, m.storageClient, executionInputs, // or request.Inputs? + workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.Inputs) + return err + }) + + var userInputsURI storage.DataReference + offloadInputsGroup.Go(func() error { + var err error + userInputsURI, err = common.OffloadLiteralMap(offloadInputsGroupCtx, m.storageClient, request.Inputs, + workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.UserInputs) + return err + }) + + err = getClosureGroup.Wait() + if err != nil { + return nil, nil, err + } + closure.CreatedAt = workflow.Closure.CreatedAt + workflow.Closure = closure + ctx = getExecutionContext(ctx, workflowExecutionID) namespace := common.GetNamespaceName( m.config.NamespaceMappingConfiguration().GetNamespaceTemplate(), workflowExecutionID.Project, workflowExecutionID.Domain) @@ -515,14 +544,6 @@ func (m *ExecutionManager) launchSingleTaskExecution( // Dynamically assign execution queues. m.populateExecutionQueue(ctx, workflow.Id, workflow.Closure.CompiledWorkflow) - inputsURI, err := common.OffloadLiteralMap(ctx, m.storageClient, request.Inputs, workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.Inputs) - if err != nil { - return nil, nil, err - } - userInputsURI, err := common.OffloadLiteralMap(ctx, m.storageClient, request.Inputs, workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.UserInputs) - if err != nil { - return nil, nil, err - } executionConfig, err := m.getExecutionConfig(ctx, request, nil) if err != nil { return nil, nil, err @@ -583,6 +604,11 @@ func (m *ExecutionManager) launchSingleTaskExecution( executionParameters.RecoveryExecution = request.Spec.Metadata.ReferenceExecution } + err = offloadInputsGroup.Wait() + if err != nil { + return nil, nil, err + } + workflowExecutor := plugins.Get[workflowengineInterfaces.WorkflowExecutor](m.pluginRegistry, plugins.PluginIDWorkflowExecutor) execInfo, err := workflowExecutor.Execute(ctx, workflowengineInterfaces.ExecutionData{ Namespace: namespace, @@ -833,13 +859,18 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( logger.Debugf(ctx, "Failed to validate ExecutionCreateRequest %+v with err %v", request, err) return nil, nil, nil, err } + if request.Spec.LaunchPlan.ResourceType == core.ResourceType_TASK { logger.Debugf(ctx, "Launching single task execution with [%+v]", request.Spec.LaunchPlan) // When tasks can have defaults this will need to handle Artifacts as well. ctx, model, err := m.launchSingleTaskExecution(ctx, request, requestedAt) return ctx, model, nil, err } + return m.launchExecution(ctx, request, requestedAt) +} +func (m *ExecutionManager) launchExecution( + ctx context.Context, request *admin.ExecutionCreateRequest, requestedAt time.Time) (context.Context, *models.Execution, []*models.ExecutionTag, error) { launchPlanModel, err := util.GetLaunchPlanModel(ctx, m.db, request.Spec.LaunchPlan) if err != nil { logger.Debugf(ctx, "Failed to get launch plan model for ExecutionCreateRequest %+v with err %v", request, err) @@ -880,13 +911,6 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( logger.Debugf(ctx, "Failed to get workflow with id %+v with err %v", launchPlan.Spec.WorkflowId, err) return nil, nil, nil, err } - closure, err := util.FetchAndGetWorkflowClosure(ctx, m.storageClient, workflowModel.RemoteClosureIdentifier) - if err != nil { - logger.Debugf(ctx, "Failed to get workflow with id %+v with err %v", launchPlan.Spec.WorkflowId, err) - return nil, nil, nil, err - } - closure.CreatedAt = workflow.Closure.CreatedAt - workflow.Closure = closure name := util.GetExecutionName(request) workflowExecutionID := &core.WorkflowExecutionIdentifier{ @@ -894,6 +918,43 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( Domain: request.Domain, Name: name, } + + // Overlap the blob store reads and writes + getClosureGroup, getClosureGroupCtx := errgroup.WithContext(ctx) + var closure *admin.WorkflowClosure + getClosureGroup.Go(func() error { + var err error + closure, err = util.FetchAndGetWorkflowClosure(getClosureGroupCtx, m.storageClient, workflowModel.RemoteClosureIdentifier) + if err != nil { + logger.Debugf(ctx, "Failed to get workflow with id %+v with err %v", launchPlan.Spec.WorkflowId, err) + } + return err + }) + + offloadInputsGroup, offloadInputsGroupCtx := errgroup.WithContext(ctx) + var inputsURI storage.DataReference + offloadInputsGroup.Go(func() error { + var err error + inputsURI, err = common.OffloadLiteralMap(offloadInputsGroupCtx, m.storageClient, executionInputs, + workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.Inputs) + return err + }) + + var userInputsURI storage.DataReference + offloadInputsGroup.Go(func() error { + var err error + userInputsURI, err = common.OffloadLiteralMap(offloadInputsGroupCtx, m.storageClient, request.Inputs, + workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.UserInputs) + return err + }) + + err = getClosureGroup.Wait() + if err != nil { + return nil, nil, nil, err + } + closure.CreatedAt = workflow.Closure.CreatedAt + workflow.Closure = closure + ctx = getExecutionContext(ctx, workflowExecutionID) var requestSpec = request.Spec if requestSpec.Metadata == nil { @@ -919,15 +980,6 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( // Dynamically assign execution queues. m.populateExecutionQueue(ctx, workflow.Id, workflow.Closure.CompiledWorkflow) - inputsURI, err := common.OffloadLiteralMap(ctx, m.storageClient, executionInputs, workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.Inputs) - if err != nil { - return nil, nil, nil, err - } - userInputsURI, err := common.OffloadLiteralMap(ctx, m.storageClient, request.Inputs, workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.UserInputs) - if err != nil { - return nil, nil, nil, err - } - executionConfig, err := m.getExecutionConfig(ctx, request, launchPlan) if err != nil { return nil, nil, nil, err @@ -1006,6 +1058,11 @@ func (m *ExecutionManager) launchExecutionAndPrepareModel( notificationsSettings = make([]*admin.Notification, 0) } + err = offloadInputsGroup.Wait() + if err != nil { + return nil, nil, nil, err + } + createExecModelInput := transformers.CreateExecutionModelInput{ WorkflowExecutionID: workflowExecutionID, RequestSpec: requestSpec, diff --git a/flyteadmin/pkg/manager/impl/execution_manager_test.go b/flyteadmin/pkg/manager/impl/execution_manager_test.go index 1cf2713083..93d327bd53 100644 --- a/flyteadmin/pkg/manager/impl/execution_manager_test.go +++ b/flyteadmin/pkg/manager/impl/execution_manager_test.go @@ -5,6 +5,7 @@ import ( "errors" "fmt" "strings" + "sync" "testing" "time" @@ -238,8 +239,11 @@ func setDefaultTaskCallbackForExecTest(repository interfaces.Repository) { func getMockStorageForExecTest(ctx context.Context) *storage.DataStore { mockStorage := commonMocks.GetMockStorageClient() + var mtx sync.RWMutex mockStorage.ComposedProtobufStore.(*commonMocks.TestDataStore).ReadProtobufCb = func( ctx context.Context, reference storage.DataReference, msg proto.Message) error { + mtx.RLock() + defer mtx.RUnlock() if val, ok := mockStorage.ComposedProtobufStore.(*commonMocks.TestDataStore).Store[reference]; ok { _ = proto.Unmarshal(val, msg) return nil @@ -252,6 +256,8 @@ func getMockStorageForExecTest(ctx context.Context) *storage.DataStore { if err != nil { return err } + mtx.Lock() + defer mtx.Unlock() mockStorage.ComposedProtobufStore.(*commonMocks.TestDataStore).Store[reference] = bytes return nil } @@ -2002,6 +2008,10 @@ func TestRecoverExecution_GetExistingInputsFailure(t *testing.T) { ctx context.Context, reference storage.DataReference, msg proto.Message) error { return expectedErr } + mockStorage.ComposedProtobufStore.(*commonMocks.TestDataStore).WriteProtobufCb = func( + ctx context.Context, reference storage.DataReference, opts storage.Options, msg proto.Message) error { + return nil + } r := plugins.NewRegistry() r.RegisterDefault(plugins.PluginIDWorkflowExecutor, &defaultTestExecutor) execManager := NewExecutionManager(repository, r, getMockExecutionsConfigProvider(), mockStorage, mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, nil, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) From 9a874fba1c8bd6cf68879528ee589fb774f5edac Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Fri, 23 Feb 2024 16:10:29 -0800 Subject: [PATCH 02/25] Overlap FutureFileReader blob store writes/reads This change updates `FutureFileReader.Cache` and `FutureFileReader.RetrieveCache` to use overlapped write and reads, respectively, to reduce end-to-end latency. The read path is a common operation on each iteration of the propeller `Handle` loop for dynamic nodes. Signed-off-by: Andrew Dye --- .../nodes/dynamic/dynamic_workflow_test.go | 5 +++ .../nodes/task/future_file_reader.go | 33 ++++++++++++------- 2 files changed, 27 insertions(+), 11 deletions(-) diff --git a/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go b/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go index 389ea0439b..3cb27dd65f 100644 --- a/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go +++ b/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go @@ -499,6 +499,11 @@ func Test_dynamicNodeHandler_buildContextualDynamicWorkflow_withLaunchPlans(t *t int64(1501), storage.Options{}, mock.MatchedBy(func(rdr *bytes.Reader) bool { return true })).Return(errors.New("foo")) + composedPBStore.OnWriteProtobufMatch( + mock.MatchedBy(func(ctx context.Context) bool { return true }), + storage.DataReference("s3://my-s3-bucket/foo/bar/dynamic_compiled.pb"), + storage.Options{}, + mock.MatchedBy(func(pb *core.CompiledWorkflowClosure) bool { return true })).Return(nil) referenceConstructor := storageMocks.ReferenceConstructor{} referenceConstructor.On("ConstructReference", mock.MatchedBy(func(ctx context.Context) bool { return true }), storage.DataReference("output-dir"), "futures.pb").Return( diff --git a/flytepropeller/pkg/controller/nodes/task/future_file_reader.go b/flytepropeller/pkg/controller/nodes/task/future_file_reader.go index fa23986812..c535ed5896 100644 --- a/flytepropeller/pkg/controller/nodes/task/future_file_reader.go +++ b/flytepropeller/pkg/controller/nodes/task/future_file_reader.go @@ -3,6 +3,8 @@ package task import ( "context" + "golang.org/x/sync/errgroup" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flyte/flytepropeller/pkg/utils" @@ -58,11 +60,14 @@ func (f FutureFileReader) CacheExists(ctx context.Context) (bool, error) { } func (f FutureFileReader) Cache(ctx context.Context, wf *v1alpha1.FlyteWorkflow, workflowClosure *core.CompiledWorkflowClosure) error { - err := f.RemoteFileWorkflowStore.PutFlyteWorkflowCRD(ctx, wf, f.flyteWfCRDCacheLoc) - if err != nil { - return err - } - return f.RemoteFileWorkflowStore.PutCompiledFlyteWorkflow(ctx, workflowClosure, f.flyteWfClosureCacheLoc) + group, ctx := errgroup.WithContext(ctx) + group.Go(func() error { + return f.RemoteFileWorkflowStore.PutFlyteWorkflowCRD(ctx, wf, f.flyteWfCRDCacheLoc) + }) + group.Go(func() error { + return f.RemoteFileWorkflowStore.PutCompiledFlyteWorkflow(ctx, workflowClosure, f.flyteWfClosureCacheLoc) + }) + return group.Wait() } type CacheContents struct { @@ -71,12 +76,18 @@ type CacheContents struct { } func (f FutureFileReader) RetrieveCache(ctx context.Context) (CacheContents, error) { - workflowCRD, err := f.RemoteFileWorkflowStore.GetWorkflowCRD(ctx, f.flyteWfCRDCacheLoc) - if err != nil { - return CacheContents{}, err - } - compiledWorkflow, err := f.RemoteFileWorkflowStore.GetCompiledWorkflow(ctx, f.flyteWfClosureCacheLoc) - if err != nil { + group, ctx := errgroup.WithContext(ctx) + var workflowCRD *v1alpha1.FlyteWorkflow + group.Go(func() (err error) { + workflowCRD, err = f.RemoteFileWorkflowStore.GetWorkflowCRD(ctx, f.flyteWfCRDCacheLoc) + return + }) + var compiledWorkflow *core.CompiledWorkflowClosure + group.Go(func() (err error) { + compiledWorkflow, err = f.RemoteFileWorkflowStore.GetCompiledWorkflow(ctx, f.flyteWfClosureCacheLoc) + return + }) + if err := group.Wait(); err != nil { return CacheContents{}, err } return CacheContents{ From ea56a9649e04840d773d761e8550b30d31f96fc2 Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Fri, 1 Mar 2024 15:43:29 -0800 Subject: [PATCH 03/25] Fix async notifications tests I didn't chase down why assumptions changed here and why these tests broke, but fixing them with more explicit checks. Signed-off-by: Andrew Dye --- .../notifications/implementations/gcp_processor_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flyteadmin/pkg/async/notifications/implementations/gcp_processor_test.go b/flyteadmin/pkg/async/notifications/implementations/gcp_processor_test.go index 5ad49a7257..d48efeeee9 100644 --- a/flyteadmin/pkg/async/notifications/implementations/gcp_processor_test.go +++ b/flyteadmin/pkg/async/notifications/implementations/gcp_processor_test.go @@ -48,7 +48,7 @@ func TestGcpProcessor_StartProcessing(t *testing.T) { m := &dto.Metric{} err := testGcpProcessor.(*GcpProcessor).systemMetrics.MessageSuccess.Write(m) assert.Nil(t, err) - assert.Equal(t, "counter:{value:1}", m.String()) + assert.Equal(t, float64(1), m.GetCounter().GetValue()) } func TestGcpProcessor_StartProcessingNoMessages(t *testing.T) { @@ -63,7 +63,7 @@ func TestGcpProcessor_StartProcessingNoMessages(t *testing.T) { m := &dto.Metric{} err := testGcpProcessor.(*GcpProcessor).systemMetrics.MessageSuccess.Write(m) assert.Nil(t, err) - assert.Equal(t, "counter:{value:0}", m.String()) + assert.Equal(t, float64(0), m.GetCounter().GetValue()) } func TestGcpProcessor_StartProcessingError(t *testing.T) { @@ -96,7 +96,7 @@ func TestGcpProcessor_StartProcessingEmailError(t *testing.T) { m := &dto.Metric{} err := testGcpProcessor.(*GcpProcessor).systemMetrics.MessageProcessorError.Write(m) assert.Nil(t, err) - assert.Equal(t, "counter:{value:1}", m.String()) + assert.Equal(t, float64(1), m.GetCounter().GetValue()) } func TestGcpProcessor_StopProcessing(t *testing.T) { From 3e1f2497fed6eac58cfe735c583288809ddfbf18 Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Sun, 3 Mar 2024 09:39:36 -0800 Subject: [PATCH 04/25] Overlap fetching input and output data This change updates `GetExecutionData`, `GetNodeExecutionData`, and `GetTaskExecutionData` to use overlapped reads when fetching input and output data. Signed-off-by: Andrew Dye --- flyteadmin/go.mod | 1 + flyteadmin/go.sum | 2 + .../pkg/manager/impl/execution_manager.go | 51 +++++++++++-------- .../manager/impl/node_execution_manager.go | 27 +++++++--- .../manager/impl/task_execution_manager.go | 28 +++++++--- 5 files changed, 74 insertions(+), 35 deletions(-) diff --git a/flyteadmin/go.mod b/flyteadmin/go.mod index 836bc69979..a7806557d7 100644 --- a/flyteadmin/go.mod +++ b/flyteadmin/go.mod @@ -165,6 +165,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/procfs v0.10.1 // indirect github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 // indirect + github.com/samber/lo v1.47.0 // indirect github.com/sendgrid/rest v2.6.9+incompatible // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/spf13/afero v1.8.2 // indirect diff --git a/flyteadmin/go.sum b/flyteadmin/go.sum index 7c9c02881f..4667cd3083 100644 --- a/flyteadmin/go.sum +++ b/flyteadmin/go.sum @@ -1156,6 +1156,8 @@ github.com/rubenv/sql-migrate v0.0.0-20190212093014-1007f53448d7/go.mod h1:WS0rl github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/samber/lo v1.47.0 h1:z7RynLwP5nbyRscyvcD043DWYoOcYRv3mV8lBeqOCLc= +github.com/samber/lo v1.47.0/go.mod h1:RmDH9Ct32Qy3gduHQuKJ3gW1fMHAnE/fAzQuf6He5cU= github.com/santhosh-tekuri/jsonschema v1.2.4/go.mod h1:TEAUOeZSmIxTTuHatJzrvARHiuO9LYd+cIxzgEHCQI4= github.com/santhosh-tekuri/jsonschema/v2 v2.1.0/go.mod h1:yzJzKUGV4RbWqWIBBP4wSOBqavX5saE02yirLS0OTyg= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= diff --git a/flyteadmin/pkg/manager/impl/execution_manager.go b/flyteadmin/pkg/manager/impl/execution_manager.go index 113c3b08a8..7c64ccbc3c 100644 --- a/flyteadmin/pkg/manager/impl/execution_manager.go +++ b/flyteadmin/pkg/manager/impl/execution_manager.go @@ -920,35 +920,34 @@ func (m *ExecutionManager) launchExecution( } // Overlap the blob store reads and writes - getClosureGroup, getClosureGroupCtx := errgroup.WithContext(ctx) + group, groupCtx := errgroup.WithContext(ctx) var closure *admin.WorkflowClosure - getClosureGroup.Go(func() error { + group.Go(func() error { var err error - closure, err = util.FetchAndGetWorkflowClosure(getClosureGroupCtx, m.storageClient, workflowModel.RemoteClosureIdentifier) + closure, err = util.FetchAndGetWorkflowClosure(groupCtx, m.storageClient, workflowModel.RemoteClosureIdentifier) if err != nil { logger.Debugf(ctx, "Failed to get workflow with id %+v with err %v", launchPlan.Spec.WorkflowId, err) } return err }) - offloadInputsGroup, offloadInputsGroupCtx := errgroup.WithContext(ctx) var inputsURI storage.DataReference - offloadInputsGroup.Go(func() error { + group.Go(func() error { var err error - inputsURI, err = common.OffloadLiteralMap(offloadInputsGroupCtx, m.storageClient, executionInputs, + inputsURI, err = common.OffloadLiteralMap(groupCtx, m.storageClient, executionInputs, workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.Inputs) return err }) var userInputsURI storage.DataReference - offloadInputsGroup.Go(func() error { + group.Go(func() error { var err error - userInputsURI, err = common.OffloadLiteralMap(offloadInputsGroupCtx, m.storageClient, request.Inputs, + userInputsURI, err = common.OffloadLiteralMap(groupCtx, m.storageClient, request.Inputs, workflowExecutionID.Project, workflowExecutionID.Domain, workflowExecutionID.Name, shared.UserInputs) return err }) - err = getClosureGroup.Wait() + err = group.Wait() if err != nil { return nil, nil, nil, err } @@ -1058,11 +1057,6 @@ func (m *ExecutionManager) launchExecution( notificationsSettings = make([]*admin.Notification, 0) } - err = offloadInputsGroup.Wait() - if err != nil { - return nil, nil, nil, err - } - createExecModelInput := transformers.CreateExecutionModelInput{ WorkflowExecutionID: workflowExecutionID, RequestSpec: requestSpec, @@ -1595,16 +1589,31 @@ func (m *ExecutionManager) GetExecutionData( return nil, err } } - inputs, inputURLBlob, err := util.GetInputs(ctx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), - m.storageClient, executionModel.InputsURI.String()) - if err != nil { - return nil, err - } - outputs, outputURLBlob, err := util.GetOutputs(ctx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), - m.storageClient, util.ToExecutionClosureInterface(execution.Closure)) + + var inputs *core.LiteralMap + var inputURLBlob *admin.UrlBlob + group, groupCtx := errgroup.WithContext(ctx) + group.Go(func() error { + var err error + inputs, inputURLBlob, err = util.GetInputs(groupCtx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), + m.storageClient, executionModel.InputsURI.String()) + return err + }) + + var outputs *core.LiteralMap + var outputURLBlob *admin.UrlBlob + group.Go(func() error { + var err error + outputs, outputURLBlob, err = util.GetOutputs(groupCtx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), + m.storageClient, util.ToExecutionClosureInterface(execution.Closure)) + return err + }) + + err = group.Wait() if err != nil { return nil, err } + response := &admin.WorkflowExecutionGetDataResponse{ Inputs: inputURLBlob, Outputs: outputURLBlob, diff --git a/flyteadmin/pkg/manager/impl/node_execution_manager.go b/flyteadmin/pkg/manager/impl/node_execution_manager.go index be498d52ae..2c6709dc6c 100644 --- a/flyteadmin/pkg/manager/impl/node_execution_manager.go +++ b/flyteadmin/pkg/manager/impl/node_execution_manager.go @@ -7,6 +7,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/prometheus/client_golang/prometheus" + "golang.org/x/sync/errgroup" "google.golang.org/grpc/codes" cloudeventInterfaces "github.com/flyteorg/flyte/flyteadmin/pkg/async/cloudevent/interfaces" @@ -520,14 +521,26 @@ func (m *NodeExecutionManager) GetNodeExecutionData( return nil, err } - inputs, inputURLBlob, err := util.GetInputs(ctx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), - m.storageClient, nodeExecution.InputUri) - if err != nil { - return nil, err - } + var inputs *core.LiteralMap + var inputURLBlob *admin.UrlBlob + group, groupCtx := errgroup.WithContext(ctx) + group.Go(func() error { + var err error + inputs, inputURLBlob, err = util.GetInputs(groupCtx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), + m.storageClient, nodeExecution.InputUri) + return err + }) + + var outputs *core.LiteralMap + var outputURLBlob *admin.UrlBlob + group.Go(func() error { + var err error + outputs, outputURLBlob, err = util.GetOutputs(groupCtx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), + m.storageClient, nodeExecution.Closure) + return err + }) - outputs, outputURLBlob, err := util.GetOutputs(ctx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), - m.storageClient, nodeExecution.Closure) + err = group.Wait() if err != nil { return nil, err } diff --git a/flyteadmin/pkg/manager/impl/task_execution_manager.go b/flyteadmin/pkg/manager/impl/task_execution_manager.go index ad5d7423b6..04811200ac 100644 --- a/flyteadmin/pkg/manager/impl/task_execution_manager.go +++ b/flyteadmin/pkg/manager/impl/task_execution_manager.go @@ -7,6 +7,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/prometheus/client_golang/prometheus" + "golang.org/x/sync/errgroup" "google.golang.org/grpc/codes" cloudeventInterfaces "github.com/flyteorg/flyte/flyteadmin/pkg/async/cloudevent/interfaces" @@ -310,13 +311,26 @@ func (m *TaskExecutionManager) GetTaskExecutionData( return nil, err } - inputs, inputURLBlob, err := util.GetInputs(ctx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), - m.storageClient, taskExecution.InputUri) - if err != nil { - return nil, err - } - outputs, outputURLBlob, err := util.GetOutputs(ctx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), - m.storageClient, taskExecution.Closure) + var inputs *core.LiteralMap + var inputURLBlob *admin.UrlBlob + group, groupCtx := errgroup.WithContext(ctx) + group.Go(func() error { + var err error + inputs, inputURLBlob, err = util.GetInputs(groupCtx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), + m.storageClient, taskExecution.InputUri) + return err + }) + + var outputs *core.LiteralMap + var outputURLBlob *admin.UrlBlob + group.Go(func() error { + var err error + outputs, outputURLBlob, err = util.GetOutputs(groupCtx, m.urlData, m.config.ApplicationConfiguration().GetRemoteDataConfig(), + m.storageClient, taskExecution.Closure) + return err + }) + + err = group.Wait() if err != nil { return nil, err } From 6f2b2fe9034b32782dce95970eb0cf61b7d2fe24 Mon Sep 17 00:00:00 2001 From: Dan Rammer Date: Wed, 6 Mar 2024 07:50:38 -0600 Subject: [PATCH 05/25] Add configuration for launchplan cache resync duration Currently, the launchplan cache resync duration uses the DownstreamEval duration configuration which is also used for the sync period on the k8s client. This means if we want to configure a more aggressive launchplan cache resync, we would also incur overhead in syncing all k8s resources (ex. Pods from `PodPlugin`). By adding a separate configuration value we can update these independently. Signed-off-by: Andrew Dye --- flytepropeller/pkg/controller/controller.go | 2 +- .../nodes/subworkflow/launchplan/admin.go | 5 +-- .../subworkflow/launchplan/admin_test.go | 41 ++++++++++++------- .../subworkflow/launchplan/adminconfig.go | 9 ++++ .../launchplan/adminconfig_flags.go | 1 + .../launchplan/adminconfig_flags_test.go | 14 +++++++ 6 files changed, 54 insertions(+), 18 deletions(-) diff --git a/flytepropeller/pkg/controller/controller.go b/flytepropeller/pkg/controller/controller.go index 39047e811d..6d4a4012b3 100644 --- a/flytepropeller/pkg/controller/controller.go +++ b/flytepropeller/pkg/controller/controller.go @@ -340,7 +340,7 @@ func New(ctx context.Context, cfg *config.Config, kubeClientset kubernetes.Inter var launchPlanActor launchplan.FlyteAdmin if cfg.EnableAdminLauncher { - launchPlanActor, err = launchplan.NewAdminLaunchPlanExecutor(ctx, adminClient, cfg.DownstreamEval.Duration, + launchPlanActor, err = launchplan.NewAdminLaunchPlanExecutor(ctx, adminClient, launchplan.GetAdminConfig(), scope.NewSubScope("admin_launcher"), store) if err != nil { logger.Errorf(ctx, "failed to create Admin workflow Launcher, err: %v", err.Error()) diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go index fbe0a8c1a6..a1e7d62577 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go @@ -3,7 +3,6 @@ package launchplan import ( "context" "fmt" - "time" "github.com/golang/protobuf/ptypes/wrappers" "golang.org/x/time/rate" @@ -321,14 +320,14 @@ func (a *adminLaunchPlanExecutor) syncItem(ctx context.Context, batch cache.Batc } func NewAdminLaunchPlanExecutor(_ context.Context, client service.AdminServiceClient, - syncPeriod time.Duration, cfg *AdminConfig, scope promutils.Scope, store *storage.DataStore) (FlyteAdmin, error) { + cfg *AdminConfig, scope promutils.Scope, store *storage.DataStore) (FlyteAdmin, error) { exec := &adminLaunchPlanExecutor{ adminClient: client, store: store, } rateLimiter := &workqueue.BucketRateLimiter{Limiter: rate.NewLimiter(rate.Limit(cfg.TPS), cfg.Burst)} - c, err := cache.NewAutoRefreshCache("admin-launcher", exec.syncItem, rateLimiter, syncPeriod, cfg.Workers, cfg.MaxCacheSize, scope) + c, err := cache.NewAutoRefreshCache("admin-launcher", exec.syncItem, rateLimiter, cfg.CacheResyncDuration.Duration, cfg.Workers, cfg.MaxCacheSize, scope) if err != nil { return nil, err } diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go index 2a442e3262..ad82489216 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go @@ -18,6 +18,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flytestdlib/cache" mocks2 "github.com/flyteorg/flyte/flytestdlib/cache/mocks" + "github.com/flyteorg/flyte/flytestdlib/config" "github.com/flyteorg/flyte/flytestdlib/contextutils" "github.com/flyteorg/flyte/flytestdlib/promutils" "github.com/flyteorg/flyte/flytestdlib/promutils/labeled" @@ -27,6 +28,9 @@ import ( func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { ctx := context.TODO() + adminConfig := defaultAdminConfig + adminConfig.CacheResyncDuration = config.Duration{Duration: time.Millisecond} + id := &core.WorkflowExecutionIdentifier{ Name: "n", Domain: "d", @@ -39,7 +43,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { t.Run("happy", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Millisecond, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) assert.NoError(t, err) mockClient.On("GetExecution", ctx, @@ -66,7 +70,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { mock.MatchedBy(func(o *admin.WorkflowExecutionGetRequest) bool { return true }), ).Return(nil, status.Error(codes.NotFound, "")) - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Millisecond, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) assert.NoError(t, err) assert.NoError(t, exec.Initialize(ctx)) @@ -112,7 +116,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { mock.MatchedBy(func(o *admin.WorkflowExecutionGetRequest) bool { return true }), ).Return(nil, status.Error(codes.Canceled, "")) - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Millisecond, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) assert.NoError(t, err) assert.NoError(t, exec.Initialize(ctx)) @@ -147,6 +151,8 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { ctx := context.TODO() + adminConfig := defaultAdminConfig + adminConfig.CacheResyncDuration = config.Duration{Duration: time.Second} id := &core.WorkflowExecutionIdentifier{ Name: "n", Domain: "d", @@ -158,7 +164,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { t.Run("happy", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) mockClient.On("CreateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionCreateRequest) bool { @@ -203,7 +209,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { Name: "orig", }, } - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) mockClient.On("RecoverExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionRecoverRequest) bool { @@ -239,7 +245,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { Name: "orig", }, } - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) assert.NoError(t, err) recoveryErr := status.Error(codes.NotFound, "foo") @@ -281,7 +287,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { t.Run("notFound", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) mockClient.On("CreateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionCreateRequest) bool { return true }), @@ -309,7 +315,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { t.Run("other", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) mockClient.On("CreateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionCreateRequest) bool { return true }), @@ -337,6 +343,8 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { ctx := context.TODO() + adminConfig := defaultAdminConfig + adminConfig.CacheResyncDuration = config.Duration{Duration: time.Second} id := &core.WorkflowExecutionIdentifier{ Name: "n", Domain: "d", @@ -349,7 +357,7 @@ func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { t.Run("happy", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) mockClient.On("TerminateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionTerminateRequest) bool { return o.Id == id && o.Cause == reason }), @@ -362,7 +370,7 @@ func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { t.Run("notFound", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) mockClient.On("TerminateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionTerminateRequest) bool { return o.Id == id && o.Cause == reason }), @@ -375,7 +383,7 @@ func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { t.Run("other", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) mockClient.On("TerminateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionTerminateRequest) bool { return o.Id == id && o.Cause == reason }), @@ -389,6 +397,8 @@ func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { func TestNewAdminLaunchPlanExecutor_GetLaunchPlan(t *testing.T) { ctx := context.TODO() + adminConfig := defaultAdminConfig + adminConfig.CacheResyncDuration = config.Duration{Duration: time.Second} id := &core.Identifier{ ResourceType: core.ResourceType_LAUNCH_PLAN, Name: "n", @@ -401,7 +411,7 @@ func TestNewAdminLaunchPlanExecutor_GetLaunchPlan(t *testing.T) { t.Run("launch plan found", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) assert.NoError(t, err) mockClient.OnGetLaunchPlanMatch( ctx, @@ -414,7 +424,7 @@ func TestNewAdminLaunchPlanExecutor_GetLaunchPlan(t *testing.T) { t.Run("launch plan not found", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Second, defaultAdminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) assert.NoError(t, err) mockClient.OnGetLaunchPlanMatch( ctx, @@ -443,6 +453,9 @@ type test struct { func TestAdminLaunchPlanExecutorScenarios(t *testing.T) { ctx := context.TODO() + adminConfig := defaultAdminConfig + adminConfig.CacheResyncDuration = config.Duration{Duration: time.Millisecond} + mockExecutionRespWithOutputs := &admin.Execution{ Closure: &admin.ExecutionClosure{ Phase: core.WorkflowExecution_SUCCEEDED, @@ -554,7 +567,7 @@ func TestAdminLaunchPlanExecutorScenarios(t *testing.T) { ComposedProtobufStore: pbStore, ReferenceConstructor: &storageMocks.ReferenceConstructor{}, } - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, time.Millisecond, defaultAdminConfig, promutils.NewTestScope(), storageClient) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), storageClient) assert.NoError(t, err) iwMock := &mocks2.ItemWrapper{} diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig.go index ffb14e3182..0c83f803af 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig.go @@ -1,7 +1,10 @@ package launchplan import ( + "time" + ctrlConfig "github.com/flyteorg/flyte/flytepropeller/pkg/controller/config" + "github.com/flyteorg/flyte/flytestdlib/config" ) //go:generate pflags AdminConfig --default-var defaultAdminConfig @@ -12,6 +15,9 @@ var ( Burst: 10, MaxCacheSize: 10000, Workers: 10, + CacheResyncDuration: config.Duration{ + Duration: 30 * time.Second, + }, } adminConfigSection = ctrlConfig.MustRegisterSubSection("admin-launcher", defaultAdminConfig) @@ -31,6 +37,9 @@ type AdminConfig struct { MaxCacheSize int `json:"cacheSize" pflag:",Maximum cache in terms of number of items stored."` Workers int `json:"workers" pflag:",Number of parallel workers to work on the queue."` + + // CacheResyncDuration defines the interval that the admin launcher should refresh the launchplan cache. + CacheResyncDuration config.Duration `json:"cache-resync-duration" pflag:",Frequency of re-syncing launchplans within the auto refresh cache."` } func GetAdminConfig() *AdminConfig { diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags.go index 3bb535e179..a0f36edb11 100755 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags.go @@ -54,5 +54,6 @@ func (cfg AdminConfig) GetPFlagSet(prefix string) *pflag.FlagSet { cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "burst"), defaultAdminConfig.Burst, "Maximum burst for throttle") cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "cacheSize"), defaultAdminConfig.MaxCacheSize, "Maximum cache in terms of number of items stored.") cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "workers"), defaultAdminConfig.Workers, "Number of parallel workers to work on the queue.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "cache-resync-duration"), defaultAdminConfig.CacheResyncDuration.String(), "Frequency of re-syncing launchplans within the auto refresh cache.") return cmdFlags } diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags_test.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags_test.go index bbff474eb1..7e4f8f4a67 100755 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags_test.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/adminconfig_flags_test.go @@ -155,4 +155,18 @@ func TestAdminConfig_SetFlags(t *testing.T) { } }) }) + t.Run("Test_cache-resync-duration", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultAdminConfig.CacheResyncDuration.String() + + cmdFlags.Set("cache-resync-duration", testValue) + if vString, err := cmdFlags.GetString("cache-resync-duration"); err == nil { + testDecodeJson_AdminConfig(t, fmt.Sprintf("%v", vString), &actual.CacheResyncDuration) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) } From 85ac321e61db2c4d6fea172a04f937cfca791bb1 Mon Sep 17 00:00:00 2001 From: Dan Rammer Date: Fri, 8 Mar 2024 14:34:44 -0600 Subject: [PATCH 06/25] Enqueue owner on launchplan terminal state This PR enqueues the owner workflow for evaluation when the launchplan auto refresh cache detects a launchplan in a terminal state. Signed-off-by: Andrew Dye --- flytepropeller/pkg/controller/controller.go | 34 +++++++++--------- .../nodes/subworkflow/handler_test.go | 6 ++++ .../nodes/subworkflow/launchplan.go | 3 +- .../nodes/subworkflow/launchplan/admin.go | 35 ++++++++++++++----- .../subworkflow/launchplan/admin_test.go | 35 +++++++++++-------- .../subworkflow/launchplan/launchplan.go | 4 ++- .../subworkflow/launchplan/mocks/executor.go | 14 ++++---- .../launchplan/mocks/flyte_admin.go | 14 ++++---- .../nodes/subworkflow/launchplan/noop.go | 5 ++- .../nodes/subworkflow/launchplan/noop_test.go | 4 ++- .../nodes/subworkflow/launchplan_test.go | 11 +++++- 11 files changed, 106 insertions(+), 59 deletions(-) diff --git a/flytepropeller/pkg/controller/controller.go b/flytepropeller/pkg/controller/controller.go index 6d4a4012b3..8d733c33a3 100644 --- a/flytepropeller/pkg/controller/controller.go +++ b/flytepropeller/pkg/controller/controller.go @@ -338,23 +338,6 @@ func New(ctx context.Context, cfg *config.Config, kubeClientset kubernetes.Inter return nil, errors.Wrapf(err, "Failed to create Metadata storage") } - var launchPlanActor launchplan.FlyteAdmin - if cfg.EnableAdminLauncher { - launchPlanActor, err = launchplan.NewAdminLaunchPlanExecutor(ctx, adminClient, - launchplan.GetAdminConfig(), scope.NewSubScope("admin_launcher"), store) - if err != nil { - logger.Errorf(ctx, "failed to create Admin workflow Launcher, err: %v", err.Error()) - return nil, err - } - - if err := launchPlanActor.Initialize(ctx); err != nil { - logger.Errorf(ctx, "failed to initialize Admin workflow Launcher, err: %v", err.Error()) - return nil, err - } - } else { - launchPlanActor = launchplan.NewFailFastLaunchPlanExecutor() - } - logger.Info(ctx, "Setting up event sink and recorder") eventSink, err := events.ConstructEventSink(ctx, events.GetConfig(ctx), scope.NewSubScope("event_sink")) if err != nil { @@ -434,6 +417,23 @@ func New(ctx context.Context, cfg *config.Config, kubeClientset kubernetes.Inter controller.levelMonitor = NewResourceLevelMonitor(scope.NewSubScope("collector"), flyteworkflowInformer.Lister()) + var launchPlanActor launchplan.FlyteAdmin + if cfg.EnableAdminLauncher { + launchPlanActor, err = launchplan.NewAdminLaunchPlanExecutor(ctx, adminClient, launchplan.GetAdminConfig(), + scope.NewSubScope("admin_launcher"), store, controller.enqueueWorkflowForNodeUpdates) + if err != nil { + logger.Errorf(ctx, "failed to create Admin workflow Launcher, err: %v", err.Error()) + return nil, err + } + + if err := launchPlanActor.Initialize(ctx); err != nil { + logger.Errorf(ctx, "failed to initialize Admin workflow Launcher, err: %v", err.Error()) + return nil, err + } + } else { + launchPlanActor = launchplan.NewFailFastLaunchPlanExecutor() + } + recoveryClient := recovery.NewClient(adminClient) nodeHandlerFactory, err := factory.NewHandlerFactory(ctx, launchPlanActor, launchPlanActor, kubeClient, kubeClientset, catalogClient, recoveryClient, &cfg.EventConfig, cfg.LiteralOffloadingConfig, cfg.ClusterID, signalClient, scope) diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/handler_test.go b/flytepropeller/pkg/controller/nodes/subworkflow/handler_test.go index bc16e648ab..ea21ce1171 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/handler_test.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/handler_test.go @@ -148,6 +148,10 @@ func TestWorkflowNodeHandler_StartNode_Launchplan(t *testing.T) { Version: "v", ResourceType: core.ResourceType_LAUNCH_PLAN, } + k8sWorkflowID := types.NamespacedName{ + Namespace: "namespace", + Name: "name", + } mockWfNode := &mocks2.ExecutableWorkflowNode{} mockWfNode.OnGetLaunchPlanRefID().Return(&v1alpha1.Identifier{ Identifier: lpID, @@ -178,6 +182,7 @@ func TestWorkflowNodeHandler_StartNode_Launchplan(t *testing.T) { }), mock.MatchedBy(func(o *core.Identifier) bool { return lpID == o }), mock.MatchedBy(func(o *core.LiteralMap) bool { return o.Literals == nil }), + mock.MatchedBy(func(o string) bool { return o == k8sWorkflowID.String() }), ).Return(nil) nCtx := createNodeContext(v1alpha1.WorkflowNodePhaseUndefined, mockNode, mockNodeStatus) @@ -203,6 +208,7 @@ func TestWorkflowNodeHandler_StartNode_Launchplan(t *testing.T) { }), mock.MatchedBy(func(o *core.Identifier) bool { return lpID == o }), mock.MatchedBy(func(o *core.LiteralMap) bool { return o.Literals == nil }), + mock.MatchedBy(func(o string) bool { return o == k8sWorkflowID.String() }), ).Return(nil) nCtx := createNodeContextV1(v1alpha1.WorkflowNodePhaseUndefined, mockNode, mockNodeStatus) diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan.go index dd09c5d5d7..60802a6486 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan.go @@ -102,7 +102,8 @@ func (l *launchPlanHandler) StartLaunchPlan(ctx context.Context, nCtx interfaces } } } - err = l.launchPlan.Launch(ctx, launchCtx, childID, nCtx.Node().GetWorkflowNode().GetLaunchPlanRefID().Identifier, nodeInputs) + err = l.launchPlan.Launch(ctx, launchCtx, childID, nCtx.Node().GetWorkflowNode().GetLaunchPlanRefID().Identifier, + nodeInputs, nCtx.NodeExecutionMetadata().GetOwnerID().String()) if err != nil { if launchplan.IsAlreadyExists(err) { logger.Infof(ctx, "Execution already exists [%s].", childID.Name) diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go index a1e7d62577..1ce0568bf6 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin.go @@ -14,6 +14,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/service" evtErr "github.com/flyteorg/flyte/flytepropeller/events/errors" + "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flyte/flytepropeller/pkg/compiler/transformers/k8s" "github.com/flyteorg/flyte/flytestdlib/cache" stdErr "github.com/flyteorg/flyte/flytestdlib/errors" @@ -32,9 +33,10 @@ func IsWorkflowTerminated(p core.WorkflowExecution_Phase) bool { // Executor for Launchplans that executes on a remote FlyteAdmin service (if configured) type adminLaunchPlanExecutor struct { - adminClient service.AdminServiceClient - cache cache.AutoRefresh - store *storage.DataStore + adminClient service.AdminServiceClient + cache cache.AutoRefresh + store *storage.DataStore + enqueueWorkflow v1alpha1.EnqueueWorkflow } type executionCacheItem struct { @@ -42,6 +44,7 @@ type executionCacheItem struct { ExecutionClosure *admin.ExecutionClosure SyncError error ExecutionOutputs *core.LiteralMap + ParentWorkflowID v1alpha1.WorkflowID } func (e executionCacheItem) IsTerminal() bool { @@ -78,8 +81,9 @@ func (a *adminLaunchPlanExecutor) handleLaunchError(ctx context.Context, isRecov } } -func (a *adminLaunchPlanExecutor) Launch(ctx context.Context, launchCtx LaunchContext, - executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap) error { +func (a *adminLaunchPlanExecutor) Launch(ctx context.Context, launchCtx LaunchContext, executionID *core.WorkflowExecutionIdentifier, + launchPlanRef *core.Identifier, inputs *core.LiteralMap, parentWorkflowID v1alpha1.WorkflowID) error { + var err error if launchCtx.RecoveryExecution != nil { _, err = a.adminClient.RecoverExecution(ctx, &admin.ExecutionRecoverRequest{ @@ -155,7 +159,7 @@ func (a *adminLaunchPlanExecutor) Launch(ctx context.Context, launchCtx LaunchCo } } - _, err = a.cache.GetOrCreate(executionID.String(), executionCacheItem{WorkflowExecutionIdentifier: *executionID}) + _, err = a.cache.GetOrCreate(executionID.String(), executionCacheItem{WorkflowExecutionIdentifier: *executionID, ParentWorkflowID: parentWorkflowID}) if err != nil { logger.Infof(ctx, "Failed to add ExecID [%v] to auto refresh cache", executionID) } @@ -262,6 +266,7 @@ func (a *adminLaunchPlanExecutor) syncItem(ctx context.Context, batch cache.Batc Item: executionCacheItem{ WorkflowExecutionIdentifier: exec.WorkflowExecutionIdentifier, SyncError: err, + ParentWorkflowID: exec.ParentWorkflowID, }, Action: cache.Update, }) @@ -292,6 +297,7 @@ func (a *adminLaunchPlanExecutor) syncItem(ctx context.Context, batch cache.Batc Item: executionCacheItem{ WorkflowExecutionIdentifier: exec.WorkflowExecutionIdentifier, SyncError: err, + ParentWorkflowID: exec.ParentWorkflowID, }, Action: cache.Update, }) @@ -311,19 +317,30 @@ func (a *adminLaunchPlanExecutor) syncItem(ctx context.Context, batch cache.Batc WorkflowExecutionIdentifier: exec.WorkflowExecutionIdentifier, ExecutionClosure: res.Closure, ExecutionOutputs: outputs, + ParentWorkflowID: exec.ParentWorkflowID, }, Action: cache.Update, }) } + // wait until all responses have been processed to enqueue parent workflows. if we do it + // prematurely, there is a chance the parent workflow evaluates before the cache is updated. + for _, itemSyncResponse := range resp { + exec := itemSyncResponse.Item.(executionCacheItem) + if exec.ExecutionClosure != nil && IsWorkflowTerminated(exec.ExecutionClosure.Phase) { + a.enqueueWorkflow(exec.ParentWorkflowID) + } + } + return resp, nil } func NewAdminLaunchPlanExecutor(_ context.Context, client service.AdminServiceClient, - cfg *AdminConfig, scope promutils.Scope, store *storage.DataStore) (FlyteAdmin, error) { + cfg *AdminConfig, scope promutils.Scope, store *storage.DataStore, enqueueWorkflow v1alpha1.EnqueueWorkflow) (FlyteAdmin, error) { exec := &adminLaunchPlanExecutor{ - adminClient: client, - store: store, + adminClient: client, + store: store, + enqueueWorkflow: enqueueWorkflow, } rateLimiter := &workqueue.BucketRateLimiter{Limiter: rate.NewLimiter(rate.Limit(cfg.TPS), cfg.Burst)} diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go index ad82489216..ead1312e17 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/admin_test.go @@ -43,7 +43,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { t.Run("happy", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) assert.NoError(t, err) mockClient.On("GetExecution", ctx, @@ -70,7 +70,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { mock.MatchedBy(func(o *admin.WorkflowExecutionGetRequest) bool { return true }), ).Return(nil, status.Error(codes.NotFound, "")) - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) assert.NoError(t, err) assert.NoError(t, exec.Initialize(ctx)) @@ -89,6 +89,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { id, &core.Identifier{}, nil, + "", ) assert.NoError(t, err) @@ -116,7 +117,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { mock.MatchedBy(func(o *admin.WorkflowExecutionGetRequest) bool { return true }), ).Return(nil, status.Error(codes.Canceled, "")) - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) assert.NoError(t, err) assert.NoError(t, exec.Initialize(ctx)) @@ -135,6 +136,7 @@ func TestAdminLaunchPlanExecutor_GetStatus(t *testing.T) { id, &core.Identifier{}, nil, + "", ) assert.NoError(t, err) @@ -164,7 +166,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { t.Run("happy", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) mockClient.On("CreateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionCreateRequest) bool { @@ -192,6 +194,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { id, &core.Identifier{}, nil, + "", ) assert.NoError(t, err) // Ensure we haven't mutated the state of the parent workflow. @@ -209,7 +212,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { Name: "orig", }, } - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) mockClient.On("RecoverExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionRecoverRequest) bool { @@ -230,6 +233,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { id, &core.Identifier{}, nil, + "", ) assert.NoError(t, err) }) @@ -245,7 +249,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { Name: "orig", }, } - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) assert.NoError(t, err) recoveryErr := status.Error(codes.NotFound, "foo") @@ -279,6 +283,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { id, &core.Identifier{}, nil, + "", ) assert.NoError(t, err) assert.True(t, createCalled) @@ -287,7 +292,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { t.Run("notFound", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) mockClient.On("CreateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionCreateRequest) bool { return true }), @@ -307,6 +312,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { id, &core.Identifier{}, nil, + "", ) assert.Error(t, err) assert.True(t, IsAlreadyExists(err)) @@ -315,7 +321,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { t.Run("other", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) mockClient.On("CreateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionCreateRequest) bool { return true }), @@ -335,6 +341,7 @@ func TestAdminLaunchPlanExecutor_Launch(t *testing.T) { id, &core.Identifier{}, nil, + "", ) assert.Error(t, err) assert.False(t, IsAlreadyExists(err)) @@ -357,7 +364,7 @@ func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { t.Run("happy", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) mockClient.On("TerminateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionTerminateRequest) bool { return o.Id == id && o.Cause == reason }), @@ -370,7 +377,7 @@ func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { t.Run("notFound", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) mockClient.On("TerminateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionTerminateRequest) bool { return o.Id == id && o.Cause == reason }), @@ -383,7 +390,7 @@ func TestAdminLaunchPlanExecutor_Kill(t *testing.T) { t.Run("other", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) mockClient.On("TerminateExecution", ctx, mock.MatchedBy(func(o *admin.ExecutionTerminateRequest) bool { return o.Id == id && o.Cause == reason }), @@ -411,7 +418,7 @@ func TestNewAdminLaunchPlanExecutor_GetLaunchPlan(t *testing.T) { t.Run("launch plan found", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) assert.NoError(t, err) mockClient.OnGetLaunchPlanMatch( ctx, @@ -424,7 +431,7 @@ func TestNewAdminLaunchPlanExecutor_GetLaunchPlan(t *testing.T) { t.Run("launch plan not found", func(t *testing.T) { mockClient := &mocks.AdminServiceClient{} - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), memStore, func(string) {}) assert.NoError(t, err) mockClient.OnGetLaunchPlanMatch( ctx, @@ -567,7 +574,7 @@ func TestAdminLaunchPlanExecutorScenarios(t *testing.T) { ComposedProtobufStore: pbStore, ReferenceConstructor: &storageMocks.ReferenceConstructor{}, } - exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), storageClient) + exec, err := NewAdminLaunchPlanExecutor(ctx, mockClient, adminConfig, promutils.NewTestScope(), storageClient, func(string) {}) assert.NoError(t, err) iwMock := &mocks2.ItemWrapper{} diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/launchplan.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/launchplan.go index f2262ce7b2..344a9144ea 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/launchplan.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/launchplan.go @@ -5,6 +5,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" ) //go:generate mockery -all -case=underscore @@ -36,7 +37,8 @@ type LaunchContext struct { // Executor interface to be implemented by the remote system that can allow workflow launching capabilities type Executor interface { // Launch start an execution of a launchplan - Launch(ctx context.Context, launchCtx LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap) error + Launch(ctx context.Context, launchCtx LaunchContext, executionID *core.WorkflowExecutionIdentifier, + launchPlanRef *core.Identifier, inputs *core.LiteralMap, parentWorkflowID v1alpha1.WorkflowID) error // GetStatus retrieves status of a LaunchPlan execution GetStatus(ctx context.Context, executionID *core.WorkflowExecutionIdentifier) (*admin.ExecutionClosure, *core.LiteralMap, error) diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/executor.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/executor.go index aa3a1bdb25..7286c9b3b7 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/executor.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/executor.go @@ -141,8 +141,8 @@ func (_m Executor_Launch) Return(_a0 error) *Executor_Launch { return &Executor_Launch{Call: _m.Call.Return(_a0)} } -func (_m *Executor) OnLaunch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap) *Executor_Launch { - c_call := _m.On("Launch", ctx, launchCtx, executionID, launchPlanRef, inputs) +func (_m *Executor) OnLaunch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap, parentWorkflowID string) *Executor_Launch { + c_call := _m.On("Launch", ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID) return &Executor_Launch{Call: c_call} } @@ -151,13 +151,13 @@ func (_m *Executor) OnLaunchMatch(matchers ...interface{}) *Executor_Launch { return &Executor_Launch{Call: c_call} } -// Launch provides a mock function with given fields: ctx, launchCtx, executionID, launchPlanRef, inputs -func (_m *Executor) Launch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap) error { - ret := _m.Called(ctx, launchCtx, executionID, launchPlanRef, inputs) +// Launch provides a mock function with given fields: ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID +func (_m *Executor) Launch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap, parentWorkflowID string) error { + ret := _m.Called(ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, launchplan.LaunchContext, *core.WorkflowExecutionIdentifier, *core.Identifier, *core.LiteralMap) error); ok { - r0 = rf(ctx, launchCtx, executionID, launchPlanRef, inputs) + if rf, ok := ret.Get(0).(func(context.Context, launchplan.LaunchContext, *core.WorkflowExecutionIdentifier, *core.Identifier, *core.LiteralMap, string) error); ok { + r0 = rf(ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID) } else { r0 = ret.Error(0) } diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/flyte_admin.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/flyte_admin.go index 3fa881ecfe..ddb887e61c 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/flyte_admin.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/mocks/flyte_admin.go @@ -182,8 +182,8 @@ func (_m FlyteAdmin_Launch) Return(_a0 error) *FlyteAdmin_Launch { return &FlyteAdmin_Launch{Call: _m.Call.Return(_a0)} } -func (_m *FlyteAdmin) OnLaunch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap) *FlyteAdmin_Launch { - c_call := _m.On("Launch", ctx, launchCtx, executionID, launchPlanRef, inputs) +func (_m *FlyteAdmin) OnLaunch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap, parentWorkflowID string) *FlyteAdmin_Launch { + c_call := _m.On("Launch", ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID) return &FlyteAdmin_Launch{Call: c_call} } @@ -192,13 +192,13 @@ func (_m *FlyteAdmin) OnLaunchMatch(matchers ...interface{}) *FlyteAdmin_Launch return &FlyteAdmin_Launch{Call: c_call} } -// Launch provides a mock function with given fields: ctx, launchCtx, executionID, launchPlanRef, inputs -func (_m *FlyteAdmin) Launch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap) error { - ret := _m.Called(ctx, launchCtx, executionID, launchPlanRef, inputs) +// Launch provides a mock function with given fields: ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID +func (_m *FlyteAdmin) Launch(ctx context.Context, launchCtx launchplan.LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap, parentWorkflowID string) error { + ret := _m.Called(ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, launchplan.LaunchContext, *core.WorkflowExecutionIdentifier, *core.Identifier, *core.LiteralMap) error); ok { - r0 = rf(ctx, launchCtx, executionID, launchPlanRef, inputs) + if rf, ok := ret.Get(0).(func(context.Context, launchplan.LaunchContext, *core.WorkflowExecutionIdentifier, *core.Identifier, *core.LiteralMap, string) error); ok { + r0 = rf(ctx, launchCtx, executionID, launchPlanRef, inputs, parentWorkflowID) } else { r0 = ret.Error(0) } diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop.go index 666d3b1797..3f7444788d 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop.go @@ -6,6 +6,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flyte/flytestdlib/errors" "github.com/flyteorg/flyte/flytestdlib/logger" ) @@ -15,7 +16,9 @@ type failFastWorkflowLauncher struct { Reader } -func (failFastWorkflowLauncher) Launch(ctx context.Context, launchCtx LaunchContext, executionID *core.WorkflowExecutionIdentifier, launchPlanRef *core.Identifier, inputs *core.LiteralMap) error { +func (failFastWorkflowLauncher) Launch(ctx context.Context, launchCtx LaunchContext, executionID *core.WorkflowExecutionIdentifier, + launchPlanRef *core.Identifier, inputs *core.LiteralMap, parentWorkflowID v1alpha1.WorkflowID) error { + logger.Infof(ctx, "Fail: Launch Workflow requested with ExecID [%s], LaunchPlan [%s]", executionID.Name, fmt.Sprintf("%s:%s:%s", launchPlanRef.Project, launchPlanRef.Domain, launchPlanRef.Name)) return errors.Wrapf(RemoteErrorUser, fmt.Errorf("badly configured system"), "please enable admin workflow launch to use launchplans") } diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop_test.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop_test.go index bf26ee0d60..c875fd3720 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop_test.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan/noop_test.go @@ -37,7 +37,9 @@ func TestFailFastWorkflowLauncher(t *testing.T) { Domain: "d", Name: "n", }, &core.Identifier{}, - nil) + nil, + "", + ) assert.Error(t, err) }) diff --git a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan_test.go b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan_test.go index 2e042d72a4..68b5383b78 100644 --- a/flytepropeller/pkg/controller/nodes/subworkflow/launchplan_test.go +++ b/flytepropeller/pkg/controller/nodes/subworkflow/launchplan_test.go @@ -52,6 +52,10 @@ func TestSubWorkflowHandler_StartLaunchPlan(t *testing.T) { Version: "v", ResourceType: core.ResourceType_LAUNCH_PLAN, } + k8sWorkflowID := types.NamespacedName{ + Namespace: "namespace", + Name: "name", + } mockWfNode := &mocks2.ExecutableWorkflowNode{} mockWfNode.On("GetLaunchPlanRefID").Return(&v1alpha1.Identifier{ Identifier: lpID, @@ -81,6 +85,7 @@ func TestSubWorkflowHandler_StartLaunchPlan(t *testing.T) { }), mock.MatchedBy(func(o *core.Identifier) bool { return lpID == o }), mock.MatchedBy(func(o *core.LiteralMap) bool { return o.Literals == nil }), + mock.MatchedBy(func(o string) bool { return o == k8sWorkflowID.String() }), ).Return(nil) wfStatus := &mocks2.MutableWorkflowNodeStatus{} @@ -110,6 +115,7 @@ func TestSubWorkflowHandler_StartLaunchPlan(t *testing.T) { }), mock.MatchedBy(func(o *core.Identifier) bool { return lpID == o }), mock.MatchedBy(func(o *core.LiteralMap) bool { return o.Literals == nil }), + mock.MatchedBy(func(o string) bool { return o == k8sWorkflowID.String() }), ).Return(errors.Wrapf(launchplan.RemoteErrorAlreadyExists, fmt.Errorf("blah"), "failed")) nCtx := createNodeContext(v1alpha1.WorkflowNodePhaseUndefined, mockNode, mockNodeStatus) @@ -136,6 +142,7 @@ func TestSubWorkflowHandler_StartLaunchPlan(t *testing.T) { }), mock.MatchedBy(func(o *core.Identifier) bool { return lpID == o }), mock.MatchedBy(func(o *core.LiteralMap) bool { return o.Literals == nil }), + mock.MatchedBy(func(o string) bool { return o == k8sWorkflowID.String() }), ).Return(errors.Wrapf(launchplan.RemoteErrorSystem, fmt.Errorf("blah"), "failed")) nCtx := createNodeContext(v1alpha1.WorkflowNodePhaseExecuting, mockNode, mockNodeStatus) @@ -162,6 +169,7 @@ func TestSubWorkflowHandler_StartLaunchPlan(t *testing.T) { }), mock.MatchedBy(func(o *core.Identifier) bool { return lpID == o }), mock.MatchedBy(func(o *core.LiteralMap) bool { return o.Literals == nil }), + mock.MatchedBy(func(o string) bool { return o == k8sWorkflowID.String() }), ).Return(errors.Wrapf(launchplan.RemoteErrorUser, fmt.Errorf("blah"), "failed")) nCtx := createNodeContext(v1alpha1.WorkflowNodePhaseExecuting, mockNode, mockNodeStatus) @@ -187,7 +195,7 @@ func TestSubWorkflowHandler_StartLaunchPlan(t *testing.T) { }, }, RecoveryExecution: recoveredExecID, - }, mock.Anything, mock.Anything, mock.Anything).Return(nil) + }, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) recoveryClient := recoveryMocks.Client{} recoveryClient.On("RecoverNodeExecution", mock.Anything, recoveredExecID, mock.Anything).Return(&admin.NodeExecution{ @@ -216,6 +224,7 @@ func TestSubWorkflowHandler_StartLaunchPlan(t *testing.T) { }), mock.MatchedBy(func(o *core.Identifier) bool { return lpID == o }), mock.MatchedBy(func(o *core.LiteralMap) bool { return o.Literals == nil }), + mock.MatchedBy(func(o string) bool { return o == k8sWorkflowID.String() }), ).Return(nil) wfStatus := &mocks2.MutableWorkflowNodeStatus{} From a72c5382878ceb11888cbcc2445b978417382404 Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Wed, 13 Mar 2024 13:00:44 -0700 Subject: [PATCH 07/25] Add client-go metrics MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Register a few metric callbacks with the client-go metrics interface so that we can monitor request latencies and rate limiting of kubeclient. ``` ❯ curl http://localhost:10254/metrics | rg k8s_client k8s_client_rate_limiter_latency_bucket{verb="GET",le="0.005"} 84 k8s_client_rate_limiter_latency_bucket{verb="GET",le="0.01"} 87 k8s_client_rate_limiter_latency_bucket{verb="GET",le="0.025"} 89 k8s_client_rate_limiter_latency_bucket{verb="GET",le="0.05"} 99 k8s_client_rate_limiter_latency_bucket{verb="GET",le="0.1"} 114 k8s_client_rate_limiter_latency_bucket{verb="GET",le="0.25"} 117 k8s_client_rate_limiter_latency_bucket{verb="GET",le="0.5"} 117 k8s_client_rate_limiter_latency_bucket{verb="GET",le="1"} 117 k8s_client_rate_limiter_latency_bucket{verb="GET",le="2.5"} 117 k8s_client_rate_limiter_latency_bucket{verb="GET",le="5"} 117 k8s_client_rate_limiter_latency_bucket{verb="GET",le="10"} 117 k8s_client_rate_limiter_latency_bucket{verb="GET",le="+Inf"} 117 k8s_client_rate_limiter_latency_sum{verb="GET"} 1.9358371670000003 k8s_client_rate_limiter_latency_count{verb="GET"} 117 k8s_client_rate_limiter_latency_bucket{verb="POST",le="0.005"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="0.01"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="0.025"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="0.05"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="0.1"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="0.25"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="0.5"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="1"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="2.5"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="5"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="10"} 6 k8s_client_rate_limiter_latency_bucket{verb="POST",le="+Inf"} 6 k8s_client_rate_limiter_latency_sum{verb="POST"} 1.0542e-05 k8s_client_rate_limiter_latency_count{verb="POST"} 6 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="0.005"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="0.01"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="0.025"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="0.05"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="0.1"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="0.25"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="0.5"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="1"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="2.5"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="5"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="10"} 1 k8s_client_rate_limiter_latency_bucket{verb="PUT",le="+Inf"} 1 k8s_client_rate_limiter_latency_sum{verb="PUT"} 5e-07 k8s_client_rate_limiter_latency_count{verb="PUT"} 1 k8s_client_request_latency_bucket{verb="GET",le="0.005"} 84 k8s_client_request_latency_bucket{verb="GET",le="0.01"} 86 k8s_client_request_latency_bucket{verb="GET",le="0.025"} 89 k8s_client_request_latency_bucket{verb="GET",le="0.05"} 99 k8s_client_request_latency_bucket{verb="GET",le="0.1"} 112 k8s_client_request_latency_bucket{verb="GET",le="0.25"} 117 k8s_client_request_latency_bucket{verb="GET",le="0.5"} 117 k8s_client_request_latency_bucket{verb="GET",le="1"} 117 k8s_client_request_latency_bucket{verb="GET",le="2.5"} 117 k8s_client_request_latency_bucket{verb="GET",le="5"} 117 k8s_client_request_latency_bucket{verb="GET",le="10"} 117 k8s_client_request_latency_bucket{verb="GET",le="+Inf"} 117 k8s_client_request_latency_sum{verb="GET"} 2.1254330859999997 k8s_client_request_latency_count{verb="GET"} 117 k8s_client_request_latency_bucket{verb="POST",le="0.005"} 5 k8s_client_request_latency_bucket{verb="POST",le="0.01"} 5 k8s_client_request_latency_bucket{verb="POST",le="0.025"} 5 k8s_client_request_latency_bucket{verb="POST",le="0.05"} 6 k8s_client_request_latency_bucket{verb="POST",le="0.1"} 6 k8s_client_request_latency_bucket{verb="POST",le="0.25"} 6 k8s_client_request_latency_bucket{verb="POST",le="0.5"} 6 k8s_client_request_latency_bucket{verb="POST",le="1"} 6 k8s_client_request_latency_bucket{verb="POST",le="2.5"} 6 k8s_client_request_latency_bucket{verb="POST",le="5"} 6 k8s_client_request_latency_bucket{verb="POST",le="10"} 6 k8s_client_request_latency_bucket{verb="POST",le="+Inf"} 6 k8s_client_request_latency_sum{verb="POST"} 0.048558582 k8s_client_request_latency_count{verb="POST"} 6 k8s_client_request_latency_bucket{verb="PUT",le="0.005"} 1 k8s_client_request_latency_bucket{verb="PUT",le="0.01"} 1 k8s_client_request_latency_bucket{verb="PUT",le="0.025"} 1 k8s_client_request_latency_bucket{verb="PUT",le="0.05"} 1 k8s_client_request_latency_bucket{verb="PUT",le="0.1"} 1 k8s_client_request_latency_bucket{verb="PUT",le="0.25"} 1 k8s_client_request_latency_bucket{verb="PUT",le="0.5"} 1 k8s_client_request_latency_bucket{verb="PUT",le="1"} 1 k8s_client_request_latency_bucket{verb="PUT",le="2.5"} 1 k8s_client_request_latency_bucket{verb="PUT",le="5"} 1 k8s_client_request_latency_bucket{verb="PUT",le="10"} 1 k8s_client_request_latency_bucket{verb="PUT",le="+Inf"} 1 k8s_client_request_latency_sum{verb="PUT"} 0.002381375 k8s_client_request_latency_count{verb="PUT"} 1 k8s_client_request_total{code="200",method="GET"} 120 k8s_client_request_total{code="200",method="PUT"} 1 k8s_client_request_total{code="409",method="POST"} 6 ``` Signed-off-by: Andrew Dye --- cmd/single/root.go | 6 ++- flyteadmin/cmd/entrypoints/serve.go | 1 + flytestdlib/promutils/client.go | 80 +++++++++++++++++++++++++++++ 3 files changed, 85 insertions(+), 2 deletions(-) create mode 100644 flytestdlib/promutils/client.go diff --git a/cmd/single/root.go b/cmd/single/root.go index 1665b1841f..348e1eb557 100644 --- a/cmd/single/root.go +++ b/cmd/single/root.go @@ -8,10 +8,12 @@ import ( "github.com/flyteorg/flyte/flytestdlib/logger" - "github.com/flyteorg/flyte/flytestdlib/config" - "github.com/flyteorg/flyte/flytestdlib/config/viper" "github.com/spf13/cobra" "github.com/spf13/pflag" + + "github.com/flyteorg/flyte/flytestdlib/config" + "github.com/flyteorg/flyte/flytestdlib/config/viper" + _ "github.com/flyteorg/flyte/flytestdlib/promutils" ) var ( diff --git a/flyteadmin/cmd/entrypoints/serve.go b/flyteadmin/cmd/entrypoints/serve.go index 794b0d6671..a5a9b98bb3 100644 --- a/flyteadmin/cmd/entrypoints/serve.go +++ b/flyteadmin/cmd/entrypoints/serve.go @@ -12,6 +12,7 @@ import ( "github.com/flyteorg/flyte/flytestdlib/logger" "github.com/flyteorg/flyte/flytestdlib/otelutils" "github.com/flyteorg/flyte/flytestdlib/profutils" + _ "github.com/flyteorg/flyte/flytestdlib/promutils" ) var pluginRegistryStore = plugins.NewAtomicRegistry(plugins.NewRegistry()) diff --git a/flytestdlib/promutils/client.go b/flytestdlib/promutils/client.go new file mode 100644 index 0000000000..79a14cfe52 --- /dev/null +++ b/flytestdlib/promutils/client.go @@ -0,0 +1,80 @@ +package promutils + +import ( + "context" + "net/url" + "time" + + "github.com/prometheus/client_golang/prometheus" + "k8s.io/client-go/tools/metrics" +) + +func init() { + requestMetrics := newRequestMetricsProvider() + rateLimiterMetrics := newRateLimiterMetricsAdapter() + metrics.Register(metrics.RegisterOpts{ + RequestLatency: &requestMetrics, + RequestResult: &requestMetrics, + RateLimiterLatency: &rateLimiterMetrics, + }) +} + +var latencyBuckets = []float64{.0005, .001, .005, .01, .025, .05, .1, .25, .5, 1, 2.5, 5, 10} + +type requestMetricsProvider struct { + requestLatency *prometheus.HistogramVec + requestResult *prometheus.CounterVec +} + +func (r *requestMetricsProvider) Observe(ctx context.Context, verb string, _ url.URL, latency time.Duration) { + r.requestLatency.WithLabelValues(verb).Observe(latency.Seconds()) +} + +func (r *requestMetricsProvider) Increment(ctx context.Context, code string, method string, _ string) { + r.requestResult.WithLabelValues(code, method).Inc() +} + +func newRequestMetricsProvider() requestMetricsProvider { + requestLatency := prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Name: "k8s_client_request_latency", + Help: "Kubernetes client request latency in seconds", + Buckets: latencyBuckets, + }, + []string{"verb"}) + prometheus.MustRegister(requestLatency) + requestResult := prometheus.NewCounterVec( + prometheus.CounterOpts{ + Name: "k8s_client_request_total", + Help: "Kubernetes client request total", + }, + []string{"code", "method"}, + ) + prometheus.MustRegister(requestResult) + return requestMetricsProvider{ + requestLatency, + requestResult, + } +} + +type rateLimiterMetricsProvider struct { + rateLimiterLatency *prometheus.HistogramVec +} + +func (r *rateLimiterMetricsProvider) Observe(ctx context.Context, verb string, _ url.URL, latency time.Duration) { + r.rateLimiterLatency.WithLabelValues(verb).Observe(latency.Seconds()) +} + +func newRateLimiterMetricsAdapter() rateLimiterMetricsProvider { + rateLimiterLatency := prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Name: "k8s_client_rate_limiter_latency", + Help: "Kubernetes client rate limiter latency in seconds", + Buckets: latencyBuckets, + }, + []string{"verb"}) + prometheus.MustRegister(rateLimiterLatency) + return rateLimiterMetricsProvider{ + rateLimiterLatency, + } +} From 39b249f439cc843c261b2d722ccde4399d89b582 Mon Sep 17 00:00:00 2001 From: Joe Eschen <126913098+squiishyy@users.noreply.github.com> Date: Fri, 15 Mar 2024 11:29:46 -0700 Subject: [PATCH 08/25] Histogram Bucket Options Add abstraction to be able to pass buckets custom defined to histogram vectors. Signed-off-by: Andrew Dye --- flytestdlib/promutils/scope.go | 31 +++++++++++++++++++++++++++++ flytestdlib/promutils/scope_test.go | 6 ++++++ 2 files changed, 37 insertions(+) diff --git a/flytestdlib/promutils/scope.go b/flytestdlib/promutils/scope.go index 64d1abaeba..8525747564 100644 --- a/flytestdlib/promutils/scope.go +++ b/flytestdlib/promutils/scope.go @@ -114,6 +114,12 @@ type SummaryOptions struct { Objectives map[float64]float64 } +// A HistogramOptions represent buckets to specify for a histogram vector when creating a new prometheus histogram +type HistogramOptions struct { + // Buckets is a list of pre-determined buckets for the histogram + Buckets []float64 +} + // A Scope represents a prefix in Prometheus. It is nestable, thus every metric that is published does not need to // provide a prefix, but just the name of the metric. As long as the Scope is used to create a new instance of the metric // The prefix (or scope) is automatically set. @@ -154,6 +160,12 @@ type Scope interface { NewHistogramVec(name, description string, labelNames ...string) (*prometheus.HistogramVec, error) MustNewHistogramVec(name, description string, labelNames ...string) *prometheus.HistogramVec + // NewHistogramVecWithOptions creates new prometheus.HistogramVec metric with the prefix as the CurrentScope + // with a custom set of options, such as of buckets. + // Refer to https://prometheus.io/docs/concepts/metric_types/ for more information + NewHistogramVecWithOptions(name, description string, options HistogramOptions, labelNames ...string) (*prometheus.HistogramVec, error) + MustNewHistogramVecWithOptions(name, description string, options HistogramOptions, labelNames ...string) *prometheus.HistogramVec + // NewCounter creates new prometheus.Counter metric with the prefix as the CurrentScope // Refer to https://prometheus.io/docs/concepts/metric_types/ for more information // Important to note, counters are not like typical counters. These are ever increasing and cumulative. @@ -311,6 +323,25 @@ func (m metricsScope) MustNewHistogramVec(name, description string, labelNames . return h } +func (m metricsScope) NewHistogramVecWithOptions(name, description string, options HistogramOptions, labelNames ...string) (*prometheus.HistogramVec, error) { + h := prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Name: m.NewScopedMetricName(name), + Help: description, + Buckets: options.Buckets, + }, + labelNames, + ) + + return h, prometheus.Register(h) +} + +func (m metricsScope) MustNewHistogramVecWithOptions(name, description string, options HistogramOptions, labelNames ...string) *prometheus.HistogramVec { + h, err := m.NewHistogramVecWithOptions(name, description, options, labelNames...) + panicIfError(err) + return h +} + func (m metricsScope) NewCounter(name, description string) (prometheus.Counter, error) { c := prometheus.NewCounter( prometheus.CounterOpts{ diff --git a/flytestdlib/promutils/scope_test.go b/flytestdlib/promutils/scope_test.go index 9d5a3071d8..325c160aff 100644 --- a/flytestdlib/promutils/scope_test.go +++ b/flytestdlib/promutils/scope_test.go @@ -82,6 +82,12 @@ func TestMetricsScope(t *testing.T) { assert.Panics(t, func() { _ = s.MustNewHistogramVec("xhv", description) }) + buckets := []float64{1.0, 2.0, 3.0, 4.0, 5.0, 6.0} + mvo := s.MustNewHistogramVecWithOptions("xho", description, HistogramOptions{Buckets: buckets}) + assert.NotNil(t, mvo) + assert.Panics(t, func() { + _ = s.MustNewHistogramVecWithOptions("xho", description, HistogramOptions{Buckets: buckets}) + }) }) t.Run("Summary", func(t *testing.T) { From 4752c1e4eeca35ec606e5f193b0825e13206bb13 Mon Sep 17 00:00:00 2001 From: Katrina Rogan Date: Mon, 18 Mar 2024 14:20:13 -0700 Subject: [PATCH 09/25] Add org to CreateUploadLocation Signed-off-by: Andrew Dye --- flyteadmin/dataproxy/service.go | 8 ++++++-- flyteadmin/dataproxy/service_test.go | 20 ++++++++++++++++++++ flyteadmin/go.mod | 3 ++- go.mod | 1 + go.sum | 2 ++ 5 files changed, 31 insertions(+), 3 deletions(-) diff --git a/flyteadmin/dataproxy/service.go b/flyteadmin/dataproxy/service.go index d61998835f..cb20075a2e 100644 --- a/flyteadmin/dataproxy/service.go +++ b/flyteadmin/dataproxy/service.go @@ -12,6 +12,7 @@ import ( "strings" "time" + "github.com/samber/lo" "google.golang.org/grpc/codes" "google.golang.org/protobuf/types/known/durationpb" "google.golang.org/protobuf/types/known/timestamppb" @@ -63,7 +64,7 @@ func (s Service) CreateUploadLocation(ctx context.Context, req *service.CreateUp // If we fall in here, that means that the full path is deterministic and we should check for existence. if len(req.Filename) > 0 && len(req.FilenameRoot) > 0 { knownLocation, err := createStorageLocation(ctx, s.dataStore, s.cfg.Upload, - req.Project, req.Domain, req.FilenameRoot, req.Filename) + req.Org, req.Project, req.Domain, req.FilenameRoot, req.Filename) if err != nil { logger.Errorf(ctx, "failed to create storage location. Error %v", err) return nil, errors.NewFlyteAdminErrorf(codes.Internal, "failed to create storage location, Error: %v", err) @@ -125,7 +126,7 @@ func (s Service) CreateUploadLocation(ctx context.Context, req *service.CreateUp prefix = base32.StdEncoding.EncodeToString(req.ContentMd5) } storagePath, err := createStorageLocation(ctx, s.dataStore, s.cfg.Upload, - req.Project, req.Domain, prefix, req.Filename) + req.Org, req.Project, req.Domain, prefix, req.Filename) if err != nil { logger.Errorf(ctx, "failed to create shardedStorageLocation. Error %v", err) return nil, errors.NewFlyteAdminErrorf(codes.Internal, "failed to create shardedStorageLocation, Error: %v", err) @@ -287,6 +288,9 @@ func (s Service) validateCreateDownloadLinkRequest(req *service.CreateDownloadLi func createStorageLocation(ctx context.Context, store *storage.DataStore, cfg config.DataProxyUploadConfig, keyParts ...string) (storage.DataReference, error) { + keyParts = lo.Filter(keyParts, func(key string, _ int) bool { + return key != "" + }) storagePath, err := store.ConstructReference(ctx, store.GetBaseContainerFQN(ctx), append([]string{cfg.StoragePrefix}, keyParts...)...) if err != nil { diff --git a/flyteadmin/dataproxy/service_test.go b/flyteadmin/dataproxy/service_test.go index 81193e106b..fb7a956a27 100644 --- a/flyteadmin/dataproxy/service_test.go +++ b/flyteadmin/dataproxy/service_test.go @@ -422,3 +422,23 @@ func TestService_Error(t *testing.T) { assert.Error(t, err, "no task executions") }) } + +func TestCreateStorageLocation(t *testing.T) { + ctx := context.TODO() + dataStore := commonMocks.GetMockStorageClient() + expectedStoragePath := storage.DataReference("s3://bucket/prefix/foo/bar/baz") + t.Run("no empty parts", func(t *testing.T) { + storagePath, err := createStorageLocation(ctx, dataStore, config.DataProxyUploadConfig{ + StoragePrefix: "prefix", + }, "foo", "bar", "baz") + assert.NoError(t, err) + assert.Equal(t, expectedStoragePath, storagePath) + }) + t.Run("with empty parts", func(t *testing.T) { + storagePath, err := createStorageLocation(ctx, dataStore, config.DataProxyUploadConfig{ + StoragePrefix: "prefix", + }, "foo", "bar", "", "baz") + assert.NoError(t, err) + assert.Equal(t, expectedStoragePath, storagePath) + }) +} diff --git a/flyteadmin/go.mod b/flyteadmin/go.mod index a7806557d7..11bce657a8 100644 --- a/flyteadmin/go.mod +++ b/flyteadmin/go.mod @@ -43,6 +43,7 @@ require ( github.com/prometheus/client_golang v1.16.0 github.com/prometheus/client_model v0.4.0 github.com/robfig/cron/v3 v3.0.0 + github.com/samber/lo v1.47.0 github.com/sendgrid/sendgrid-go v3.10.0+incompatible github.com/spf13/cobra v1.7.0 github.com/spf13/pflag v1.0.5 @@ -52,6 +53,7 @@ require ( go.opentelemetry.io/otel v1.24.0 golang.org/x/net v0.27.0 golang.org/x/oauth2 v0.16.0 + golang.org/x/sync v0.7.0 golang.org/x/time v0.5.0 google.golang.org/api v0.155.0 google.golang.org/genproto v0.0.0-20240123012728-ef4313101c80 @@ -191,7 +193,6 @@ require ( go.opentelemetry.io/proto/otlp v1.1.0 // indirect golang.org/x/crypto v0.25.0 // indirect golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect - golang.org/x/sync v0.7.0 // indirect golang.org/x/sys v0.22.0 // indirect golang.org/x/term v0.22.0 // indirect golang.org/x/text v0.16.0 // indirect diff --git a/go.mod b/go.mod index 6c25974da0..16fa54c50c 100644 --- a/go.mod +++ b/go.mod @@ -164,6 +164,7 @@ require ( github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1 // indirect github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 // indirect github.com/robfig/cron/v3 v3.0.0 // indirect + github.com/samber/lo v1.47.0 // indirect github.com/sendgrid/rest v2.6.9+incompatible // indirect github.com/sendgrid/sendgrid-go v3.10.0+incompatible // indirect github.com/sirupsen/logrus v1.9.3 // indirect diff --git a/go.sum b/go.sum index ac7b9f5987..813ac168a1 100644 --- a/go.sum +++ b/go.sum @@ -1192,6 +1192,8 @@ github.com/rubenv/sql-migrate v0.0.0-20190212093014-1007f53448d7/go.mod h1:WS0rl github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/samber/lo v1.47.0 h1:z7RynLwP5nbyRscyvcD043DWYoOcYRv3mV8lBeqOCLc= +github.com/samber/lo v1.47.0/go.mod h1:RmDH9Ct32Qy3gduHQuKJ3gW1fMHAnE/fAzQuf6He5cU= github.com/santhosh-tekuri/jsonschema v1.2.4/go.mod h1:TEAUOeZSmIxTTuHatJzrvARHiuO9LYd+cIxzgEHCQI4= github.com/santhosh-tekuri/jsonschema/v2 v2.1.0/go.mod h1:yzJzKUGV4RbWqWIBBP4wSOBqavX5saE02yirLS0OTyg= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= From 4996328821ee8f561645efb6dbd74827f661cff8 Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Fri, 22 Mar 2024 07:59:18 -0700 Subject: [PATCH 10/25] Add config for grpc MaxMessageSizeBytes We need to make the grpc max recv message size in propeller's admin client configurable to match the server-side configuration we support in admin. Signed-off-by: Andrew Dye --- flyteidl/clients/go/admin/client.go | 4 ++++ flyteidl/clients/go/admin/config.go | 1 + flyteidl/clients/go/admin/config_flags.go | 1 + flyteidl/clients/go/admin/config_flags_test.go | 14 ++++++++++++++ 4 files changed, 20 insertions(+) diff --git a/flyteidl/clients/go/admin/client.go b/flyteidl/clients/go/admin/client.go index 757f25b160..9f14d49dee 100644 --- a/flyteidl/clients/go/admin/client.go +++ b/flyteidl/clients/go/admin/client.go @@ -80,6 +80,10 @@ func GetAdditionalAdminClientConfigOptions(cfg *Config) []grpc.DialOption { // ever has those endpoints opts = append(opts, grpc.WithChainUnaryInterceptor(grpcPrometheus.UnaryClientInterceptor, retryInterceptor)) + if cfg.MaxMessageSizeBytes > 0 { + opts = append(opts, grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(cfg.MaxMessageSizeBytes))) + } + return opts } diff --git a/flyteidl/clients/go/admin/config.go b/flyteidl/clients/go/admin/config.go index 03f2f8ecc2..6c8adf3553 100644 --- a/flyteidl/clients/go/admin/config.go +++ b/flyteidl/clients/go/admin/config.go @@ -45,6 +45,7 @@ type Config struct { MaxBackoffDelay config.Duration `json:"maxBackoffDelay" pflag:",Max delay for grpc backoff"` PerRetryTimeout config.Duration `json:"perRetryTimeout" pflag:",gRPC per retry timeout"` MaxRetries int `json:"maxRetries" pflag:",Max number of gRPC retries"` + MaxMessageSizeBytes int `json:"maxMessageSizeBytes" pflag:",The max size in bytes for incoming gRPC messages"` AuthType AuthType `json:"authType" pflag:",Type of OAuth2 flow used for communicating with admin.ClientSecret,Pkce,ExternalCommand are valid values"` TokenRefreshWindow config.Duration `json:"tokenRefreshWindow" pflag:",Max duration between token refresh attempt and token expiry."` // Deprecated: settings will be discovered dynamically diff --git a/flyteidl/clients/go/admin/config_flags.go b/flyteidl/clients/go/admin/config_flags.go index db1305c0b1..c344147c8c 100755 --- a/flyteidl/clients/go/admin/config_flags.go +++ b/flyteidl/clients/go/admin/config_flags.go @@ -57,6 +57,7 @@ func (cfg Config) GetPFlagSet(prefix string) *pflag.FlagSet { cmdFlags.String(fmt.Sprintf("%v%v", prefix, "maxBackoffDelay"), defaultConfig.MaxBackoffDelay.String(), "Max delay for grpc backoff") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "perRetryTimeout"), defaultConfig.PerRetryTimeout.String(), "gRPC per retry timeout") cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "maxRetries"), defaultConfig.MaxRetries, "Max number of gRPC retries") + cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "maxMessageSizeBytes"), defaultConfig.MaxMessageSizeBytes, "The max size in bytes for incoming gRPC messages") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "authType"), defaultConfig.AuthType.String(), "Type of OAuth2 flow used for communicating with admin.ClientSecret, Pkce, ExternalCommand are valid values") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "tokenRefreshWindow"), defaultConfig.TokenRefreshWindow.String(), "Max duration between token refresh attempt and token expiry.") cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "useAuth"), defaultConfig.DeprecatedUseAuth, "Deprecated: Auth will be enabled/disabled based on admin's dynamically discovered information.") diff --git a/flyteidl/clients/go/admin/config_flags_test.go b/flyteidl/clients/go/admin/config_flags_test.go index e815bcb5f3..a79467dc16 100755 --- a/flyteidl/clients/go/admin/config_flags_test.go +++ b/flyteidl/clients/go/admin/config_flags_test.go @@ -197,6 +197,20 @@ func TestConfig_SetFlags(t *testing.T) { } }) }) + t.Run("Test_maxMessageSizeBytes", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("maxMessageSizeBytes", testValue) + if vInt, err := cmdFlags.GetInt("maxMessageSizeBytes"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vInt), &actual.MaxMessageSizeBytes) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) t.Run("Test_authType", func(t *testing.T) { t.Run("Override", func(t *testing.T) { From 0b935a6e06adda2b9ef2d7d061603c2fefb2c087 Mon Sep 17 00:00:00 2001 From: Michael Barrientos Date: Mon, 25 Mar 2024 23:55:08 -0700 Subject: [PATCH 11/25] Move storage cache settings to correct location Signed-off-by: Andrew Dye --- charts/flyte-core/README.md | 3 +-- charts/flyte-core/templates/_helpers.tpl | 3 +++ .../templates/propeller/configmap.yaml | 5 ----- charts/flyte-core/values-eks.yaml | 3 ++- ...loak-idp-flyteclients-without-browser.yaml | 3 ++- charts/flyte-core/values.yaml | 4 +++- charts/flyte/README.md | 4 ++-- charts/flyte/values.yaml | 4 +++- .../flyte_aws_scheduler_helm_generated.yaml | 16 +++++++++------ .../flyte_helm_controlplane_generated.yaml | 12 ++++++++--- .../eks/flyte_helm_dataplane_generated.yaml | 6 ++---- deployment/eks/flyte_helm_generated.yaml | 18 ++++++++++------- .../flyte_helm_controlplane_generated.yaml | 12 ++++++++--- .../gcp/flyte_helm_dataplane_generated.yaml | 8 +++----- deployment/gcp/flyte_helm_generated.yaml | 20 +++++++++++-------- deployment/sandbox/flyte_helm_generated.yaml | 18 ++++++++++------- .../manifests/complete-agent.yaml | 4 ++-- .../sandbox-bundled/manifests/complete.yaml | 4 ++-- docker/sandbox-bundled/manifests/dev.yaml | 4 ++-- 19 files changed, 89 insertions(+), 62 deletions(-) diff --git a/charts/flyte-core/README.md b/charts/flyte-core/README.md index ba11205264..7727eb34c2 100644 --- a/charts/flyte-core/README.md +++ b/charts/flyte-core/README.md @@ -230,7 +230,6 @@ helm install gateway bitnami/contour -n flyte | flytepropeller.additionalVolumeMounts | list | `[]` | Appends additional volume mounts to the main container's spec. May include template values. | | flytepropeller.additionalVolumes | list | `[]` | Appends additional volumes to the deployment spec. May include template values. | | flytepropeller.affinity | object | `{}` | affinity for Flytepropeller deployment | -| flytepropeller.cacheSizeMbs | int | `0` | | | flytepropeller.clusterName | string | `""` | Defines the cluster name used in events sent to Admin | | flytepropeller.configPath | string | `"/etc/flyte/config/*.yaml"` | Default regex string for searching configuration files | | flytepropeller.createCRDs | bool | `true` | Whether to install the flyteworkflows CRD with helm | @@ -292,7 +291,7 @@ helm install gateway bitnami/contour -n flyte | sparkoperator.enabled | bool | `false` | - enable or disable Sparkoperator deployment installation | | sparkoperator.plugin_config | object | `{"plugins":{"spark":{"spark-config-default":[{"spark.hadoop.fs.s3a.aws.credentials.provider":"com.amazonaws.auth.DefaultAWSCredentialsProviderChain"},{"spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version":"2"},{"spark.kubernetes.allocation.batch.size":"50"},{"spark.hadoop.fs.s3a.acl.default":"BucketOwnerFullControl"},{"spark.hadoop.fs.s3n.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem"},{"spark.hadoop.fs.AbstractFileSystem.s3n.impl":"org.apache.hadoop.fs.s3a.S3A"},{"spark.hadoop.fs.s3.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem"},{"spark.hadoop.fs.AbstractFileSystem.s3.impl":"org.apache.hadoop.fs.s3a.S3A"},{"spark.hadoop.fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem"},{"spark.hadoop.fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A"},{"spark.hadoop.fs.s3a.multipart.threshold":"536870912"},{"spark.blacklist.enabled":"true"},{"spark.blacklist.timeout":"5m"},{"spark.task.maxfailures":"8"}]}}}` | Spark plugin configuration | | sparkoperator.plugin_config.plugins.spark.spark-config-default | list | `[{"spark.hadoop.fs.s3a.aws.credentials.provider":"com.amazonaws.auth.DefaultAWSCredentialsProviderChain"},{"spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version":"2"},{"spark.kubernetes.allocation.batch.size":"50"},{"spark.hadoop.fs.s3a.acl.default":"BucketOwnerFullControl"},{"spark.hadoop.fs.s3n.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem"},{"spark.hadoop.fs.AbstractFileSystem.s3n.impl":"org.apache.hadoop.fs.s3a.S3A"},{"spark.hadoop.fs.s3.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem"},{"spark.hadoop.fs.AbstractFileSystem.s3.impl":"org.apache.hadoop.fs.s3a.S3A"},{"spark.hadoop.fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem"},{"spark.hadoop.fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A"},{"spark.hadoop.fs.s3a.multipart.threshold":"536870912"},{"spark.blacklist.enabled":"true"},{"spark.blacklist.timeout":"5m"},{"spark.task.maxfailures":"8"}]` | Spark default configuration | -| storage | object | `{"bucketName":"my-s3-bucket","custom":{},"enableMultiContainer":false,"gcs":null,"limits":{"maxDownloadMBs":10},"s3":{"accessKey":"","authType":"iam","region":"us-east-1","secretKey":""},"type":"sandbox"}` | ---------------------------------------------------- STORAGE SETTINGS | +| storage | object | `{"bucketName":"my-s3-bucket","cache":{"maxSizeMBs":0,"targetGCPercent":70},"custom":{},"enableMultiContainer":false,"gcs":null,"limits":{"maxDownloadMBs":10},"s3":{"accessKey":"","authType":"iam","region":"us-east-1","secretKey":""},"type":"sandbox"}` | ---------------------------------------------------- STORAGE SETTINGS | | storage.bucketName | string | `"my-s3-bucket"` | bucketName defines the storage bucket flyte will use. Required for all types except for sandbox. | | storage.custom | object | `{}` | Settings for storage type custom. See https://github.com/graymeta/stow for supported storage providers/settings. | | storage.enableMultiContainer | bool | `false` | toggles multi-container storage config | diff --git a/charts/flyte-core/templates/_helpers.tpl b/charts/flyte-core/templates/_helpers.tpl index f7b50c0b29..87a7615177 100755 --- a/charts/flyte-core/templates/_helpers.tpl +++ b/charts/flyte-core/templates/_helpers.tpl @@ -237,4 +237,7 @@ storage: enable-multicontainer: {{ .Values.storage.enableMultiContainer }} limits: maxDownloadMBs: {{ .Values.storage.limits.maxDownloadMBs }} + cache: + max_size_mbs: {{ .Values.storage.cache.maxSizeMBs }} + target_gc_percent: {{ .Values.storage.cache.targetGCPercent }} {{- end }} diff --git a/charts/flyte-core/templates/propeller/configmap.yaml b/charts/flyte-core/templates/propeller/configmap.yaml index 3e1b94ef2c..cceb748274 100644 --- a/charts/flyte-core/templates/propeller/configmap.yaml +++ b/charts/flyte-core/templates/propeller/configmap.yaml @@ -47,11 +47,6 @@ data: {{- end }} {{- end }} storage.yaml: | {{ tpl (include "storage" .) $ | nindent 4 }} - cache.yaml: | - storage: - cache: - max_size_mbs: {{ .Values.flytepropeller.cacheSizeMbs }} - target_gc_percent: 70 {{- with .Values.configmap.task_logs }} task_logs.yaml: | {{ tpl (toYaml .) $ | nindent 4 }} {{- end }} diff --git a/charts/flyte-core/values-eks.yaml b/charts/flyte-core/values-eks.yaml index 5a1cc1b94d..904a30bc78 100644 --- a/charts/flyte-core/values-eks.yaml +++ b/charts/flyte-core/values-eks.yaml @@ -102,7 +102,6 @@ flytepropeller: cpu: 1 ephemeral-storage: 1Gi memory: 2Gi - cacheSizeMbs: 1024 # -- Sets priorityClassName for propeller pod(s). priorityClassName: "system-cluster-critical" affinity: @@ -191,6 +190,8 @@ storage: bucketName: "{{ .Values.userSettings.bucketName }}" s3: region: "{{ .Values.userSettings.accountRegion }}" + cache: + maxSizeMBs: 1024 db: datacatalog: diff --git a/charts/flyte-core/values-keycloak-idp-flyteclients-without-browser.yaml b/charts/flyte-core/values-keycloak-idp-flyteclients-without-browser.yaml index 961611b56c..1e088ea4b7 100644 --- a/charts/flyte-core/values-keycloak-idp-flyteclients-without-browser.yaml +++ b/charts/flyte-core/values-keycloak-idp-flyteclients-without-browser.yaml @@ -215,7 +215,6 @@ flytepropeller: cpu: 10m ephemeral-storage: 50Mi memory: 100Mi - cacheSizeMbs: 0 # -- Default regex string for searching configuration files configPath: /etc/flyte/config/*.yaml @@ -399,6 +398,8 @@ storage: # -- default limits being applied to storage config limits: maxDownloadMBs: 10 + cache: + maxSizeMBs: 0 # Database configuration(These are the values for a pgdb instance with hostname of postgres-flyte and postgres/password creds) db: diff --git a/charts/flyte-core/values.yaml b/charts/flyte-core/values.yaml index 7442222e9b..fef9fc4d67 100755 --- a/charts/flyte-core/values.yaml +++ b/charts/flyte-core/values.yaml @@ -319,7 +319,6 @@ flytepropeller: cpu: 10m ephemeral-storage: 50Mi memory: 100Mi - cacheSizeMbs: 0 # -- Error reporting terminationMessagePolicy: FallbackToLogsOnError # -- Default regex string for searching configuration files @@ -610,6 +609,9 @@ storage: # -- default limits being applied to storage config limits: maxDownloadMBs: 10 + cache: + maxSizeMBs: 0 + targetGCPercent: 70 # Database configuration db: diff --git a/charts/flyte/README.md b/charts/flyte/README.md index 1c0d00661c..895be7d7b1 100644 --- a/charts/flyte/README.md +++ b/charts/flyte/README.md @@ -71,7 +71,7 @@ helm upgrade -f values-sandbox.yaml flyte . | contour.tolerations | list | `[]` | tolerations for Contour deployment | | daskoperator | object | `{"enabled":false}` | Optional: Dask Plugin using the Dask Operator | | daskoperator.enabled | bool | `false` | - enable or disable the dask operator deployment installation | -| flyte | object | `{"cluster_resource_manager":{"config":{"cluster_resources":{"customData":[{"production":[{"projectQuotaCpu":{"value":"5"}},{"projectQuotaMemory":{"value":"4000Mi"}}]},{"staging":[{"projectQuotaCpu":{"value":"2"}},{"projectQuotaMemory":{"value":"3000Mi"}}]},{"development":[{"projectQuotaCpu":{"value":"4"}},{"projectQuotaMemory":{"value":"3000Mi"}}]}],"refresh":"5m","refreshInterval":"5m","standaloneDeployment":false,"templatePath":"/etc/flyte/clusterresource/templates"}},"enabled":true,"service_account_name":"flyteadmin","templates":[{"key":"aa_namespace","value":"apiVersion: v1\nkind: Namespace\nmetadata:\n name: {{ namespace }}\nspec:\n finalizers:\n - kubernetes\n"},{"key":"ab_project_resource_quota","value":"apiVersion: v1\nkind: ResourceQuota\nmetadata:\n name: project-quota\n namespace: {{ namespace }}\nspec:\n hard:\n limits.cpu: {{ projectQuotaCpu }}\n limits.memory: {{ projectQuotaMemory }}\n"}]},"common":{"databaseSecret":{"name":"","secretManifest":{}},"flyteNamespaceTemplate":{"enabled":false},"ingress":{"albSSLRedirect":false,"annotations":{"nginx.ingress.kubernetes.io/app-root":"/console"},"enabled":true,"host":"","separateGrpcIngress":false,"separateGrpcIngressAnnotations":{"nginx.ingress.kubernetes.io/backend-protocol":"GRPC"},"tls":{"enabled":false},"webpackHMR":true}},"configmap":{"adminServer":{"auth":{"appAuth":{"thirdPartyConfig":{"flyteClient":{"clientId":"flytectl","redirectUri":"http://localhost:53593/callback","scopes":["offline","all"]}}},"authorizedUris":["https://localhost:30081","http://flyteadmin:80","http://flyteadmin.flyte.svc.cluster.local:80"],"userAuth":{"openId":{"baseUrl":"https://accounts.google.com","clientId":"657465813211-6eog7ek7li5k7i7fvgv2921075063hpe.apps.googleusercontent.com","scopes":["profile","openid"]}}},"flyteadmin":{"eventVersion":2,"metadataStoragePrefix":["metadata","admin"],"metricsScope":"flyte:","profilerPort":10254,"roleNameKey":"iam.amazonaws.com/role","testing":{"host":"http://flyteadmin"}},"server":{"grpc":{"port":8089},"httpPort":8088,"security":{"allowCors":true,"allowedHeaders":["Content-Type","flyte-authorization"],"allowedOrigins":["*"],"secure":false,"useAuth":false}}},"catalog":{"catalog-cache":{"endpoint":"datacatalog:89","insecure":true,"type":"datacatalog"}},"console":{"BASE_URL":"/console","CONFIG_DIR":"/etc/flyte/config"},"copilot":{"plugins":{"k8s":{"co-pilot":{"image":"cr.flyte.org/flyteorg/flytecopilot:v1.13.1","name":"flyte-copilot-","start-timeout":"30s"}}}},"core":{"propeller":{"downstream-eval-duration":"30s","enable-admin-launcher":true,"leader-election":{"enabled":true,"lease-duration":"15s","lock-config-map":{"name":"propeller-leader","namespace":"flyte"},"renew-deadline":"10s","retry-period":"2s"},"limit-namespace":"all","max-workflow-retries":30,"metadata-prefix":"metadata/propeller","metrics-prefix":"flyte","prof-port":10254,"queue":{"batch-size":-1,"batching-interval":"2s","queue":{"base-delay":"5s","capacity":1000,"max-delay":"120s","rate":100,"type":"maxof"},"sub-queue":{"capacity":100,"rate":10,"type":"bucket"},"type":"batch"},"rawoutput-prefix":"s3://my-s3-bucket/","workers":4,"workflow-reeval-duration":"30s"},"webhook":{"certDir":"/etc/webhook/certs","serviceName":"flyte-pod-webhook"}},"datacatalogServer":{"application":{"grpcPort":8089,"grpcServerReflection":true,"httpPort":8080},"datacatalog":{"metrics-scope":"datacatalog","profiler-port":10254,"storage-prefix":"metadata/datacatalog"}},"domain":{"domains":[{"id":"development","name":"development"},{"id":"staging","name":"staging"},{"id":"production","name":"production"}]},"enabled_plugins":{"tasks":{"task-plugins":{"default-for-task-types":{"container":"container","container_array":"k8s-array","sensor":"agent-service","sidecar":"sidecar"},"enabled-plugins":["container","sidecar","k8s-array","agent-service","echo"]}}},"k8s":{"plugins":{"k8s":{"default-cpus":"100m","default-env-from-configmaps":[],"default-env-from-secrets":[],"default-env-vars":[{"FLYTE_AWS_ENDPOINT":"http://minio.flyte:9000"},{"FLYTE_AWS_ACCESS_KEY_ID":"minio"},{"FLYTE_AWS_SECRET_ACCESS_KEY":"miniostorage"}],"default-memory":"200Mi"}}},"logger":{"logger":{"level":5,"show-source":true}},"remoteData":{"remoteData":{"region":"us-east-1","scheme":"local","signedUrls":{"durationMinutes":3}}},"resource_manager":{"propeller":{"resourcemanager":{"redis":null,"type":"noop"}}},"task_logs":{"plugins":{"logs":{"cloudwatch-enabled":false,"kubernetes-enabled":true,"kubernetes-template-uri":"http://localhost:30082/#/log/{{ \"{{\" }} .namespace {{ \"}}\" }}/{{ \"{{\" }} .podName {{ \"}}\" }}/pod?namespace={{ \"{{\" }} .namespace {{ \"}}\" }}"}}},"task_resource_defaults":{"task_resources":{"defaults":{"cpu":"100m","memory":"200Mi","storage":"5Mi"},"limits":{"cpu":2,"gpu":1,"memory":"1Gi","storage":"20Mi"}}}},"datacatalog":{"affinity":{},"configPath":"/etc/datacatalog/config/*.yaml","image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/datacatalog","tag":"v1.13.1"},"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"500m","ephemeral-storage":"100Mi","memory":"500Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"service":{"annotations":{"projectcontour.io/upstream-protocol.h2c":"grpc"},"type":"NodePort"},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"db":{"admin":{"database":{"dbname":"flyteadmin","host":"postgres","port":5432,"username":"postgres"}},"datacatalog":{"database":{"dbname":"datacatalog","host":"postgres","port":5432,"username":"postgres"}}},"deployRedoc":true,"flyteadmin":{"additionalVolumeMounts":[],"additionalVolumes":[],"affinity":{},"configPath":"/etc/flyte/config/*.yaml","env":[],"image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flyteadmin","tag":"v1.13.1"},"initialProjects":["flytesnacks","flytetester","flyteexamples"],"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"250m","ephemeral-storage":"100Mi","memory":"500Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"secrets":{},"service":{"annotations":{"projectcontour.io/upstream-protocol.h2c":"grpc"},"loadBalancerSourceRanges":[],"type":"ClusterIP"},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"flyteconsole":{"affinity":{},"ga":{"enabled":true,"tracking_id":"G-0QW4DJWJ20"},"image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flyteconsole","tag":"v1.17.1"},"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"500m","memory":"275Mi"},"requests":{"cpu":"10m","memory":"250Mi"}},"service":{"annotations":{},"type":"ClusterIP"},"tolerations":[]},"flytepropeller":{"affinity":{},"cacheSizeMbs":0,"configPath":"/etc/flyte/config/*.yaml","image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flytepropeller","tag":"v1.13.1"},"manager":false,"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"200m","ephemeral-storage":"100Mi","memory":"200Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"flytescheduler":{"affinity":{},"configPath":"/etc/flyte/config/*.yaml","image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flytescheduler","tag":"v1.13.1"},"nodeSelector":{},"podAnnotations":{},"resources":{"limits":{"cpu":"250m","ephemeral-storage":"100Mi","memory":"500Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"secrets":{},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"storage":{"bucketName":"my-s3-bucket","custom":{},"gcs":null,"s3":{"region":"us-east-1"},"type":"sandbox"},"webhook":{"enabled":true,"service":{"annotations":{"projectcontour.io/upstream-protocol.h2c":"grpc"},"type":"ClusterIP"},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]}},"workflow_notifications":{"config":{},"enabled":false},"workflow_scheduler":{"enabled":true,"type":"native"}}` | ------------------------------------------------------------------- Core System settings This section consists of Core components of Flyte and their deployment settings. This includes FlyteAdmin service, Datacatalog, FlytePropeller and Flyteconsole | +| flyte | object | `{"cluster_resource_manager":{"config":{"cluster_resources":{"customData":[{"production":[{"projectQuotaCpu":{"value":"5"}},{"projectQuotaMemory":{"value":"4000Mi"}}]},{"staging":[{"projectQuotaCpu":{"value":"2"}},{"projectQuotaMemory":{"value":"3000Mi"}}]},{"development":[{"projectQuotaCpu":{"value":"4"}},{"projectQuotaMemory":{"value":"3000Mi"}}]}],"refresh":"5m","refreshInterval":"5m","standaloneDeployment":false,"templatePath":"/etc/flyte/clusterresource/templates"}},"enabled":true,"service_account_name":"flyteadmin","templates":[{"key":"aa_namespace","value":"apiVersion: v1\nkind: Namespace\nmetadata:\n name: {{ namespace }}\nspec:\n finalizers:\n - kubernetes\n"},{"key":"ab_project_resource_quota","value":"apiVersion: v1\nkind: ResourceQuota\nmetadata:\n name: project-quota\n namespace: {{ namespace }}\nspec:\n hard:\n limits.cpu: {{ projectQuotaCpu }}\n limits.memory: {{ projectQuotaMemory }}\n"}]},"common":{"databaseSecret":{"name":"","secretManifest":{}},"flyteNamespaceTemplate":{"enabled":false},"ingress":{"albSSLRedirect":false,"annotations":{"nginx.ingress.kubernetes.io/app-root":"/console"},"enabled":true,"host":"","separateGrpcIngress":false,"separateGrpcIngressAnnotations":{"nginx.ingress.kubernetes.io/backend-protocol":"GRPC"},"tls":{"enabled":false},"webpackHMR":true}},"configmap":{"adminServer":{"auth":{"appAuth":{"thirdPartyConfig":{"flyteClient":{"clientId":"flytectl","redirectUri":"http://localhost:53593/callback","scopes":["offline","all"]}}},"authorizedUris":["https://localhost:30081","http://flyteadmin:80","http://flyteadmin.flyte.svc.cluster.local:80"],"userAuth":{"openId":{"baseUrl":"https://accounts.google.com","clientId":"657465813211-6eog7ek7li5k7i7fvgv2921075063hpe.apps.googleusercontent.com","scopes":["profile","openid"]}}},"flyteadmin":{"eventVersion":2,"metadataStoragePrefix":["metadata","admin"],"metricsScope":"flyte:","profilerPort":10254,"roleNameKey":"iam.amazonaws.com/role","testing":{"host":"http://flyteadmin"}},"server":{"grpc":{"port":8089},"httpPort":8088,"security":{"allowCors":true,"allowedHeaders":["Content-Type","flyte-authorization"],"allowedOrigins":["*"],"secure":false,"useAuth":false}}},"catalog":{"catalog-cache":{"endpoint":"datacatalog:89","insecure":true,"type":"datacatalog"}},"console":{"BASE_URL":"/console","CONFIG_DIR":"/etc/flyte/config"},"copilot":{"plugins":{"k8s":{"co-pilot":{"image":"cr.flyte.org/flyteorg/flytecopilot:v1.13.1","name":"flyte-copilot-","start-timeout":"30s"}}}},"core":{"propeller":{"downstream-eval-duration":"30s","enable-admin-launcher":true,"leader-election":{"enabled":true,"lease-duration":"15s","lock-config-map":{"name":"propeller-leader","namespace":"flyte"},"renew-deadline":"10s","retry-period":"2s"},"limit-namespace":"all","max-workflow-retries":30,"metadata-prefix":"metadata/propeller","metrics-prefix":"flyte","prof-port":10254,"queue":{"batch-size":-1,"batching-interval":"2s","queue":{"base-delay":"5s","capacity":1000,"max-delay":"120s","rate":100,"type":"maxof"},"sub-queue":{"capacity":100,"rate":10,"type":"bucket"},"type":"batch"},"rawoutput-prefix":"s3://my-s3-bucket/","workers":4,"workflow-reeval-duration":"30s"},"webhook":{"certDir":"/etc/webhook/certs","serviceName":"flyte-pod-webhook"}},"datacatalogServer":{"application":{"grpcPort":8089,"grpcServerReflection":true,"httpPort":8080},"datacatalog":{"metrics-scope":"datacatalog","profiler-port":10254,"storage-prefix":"metadata/datacatalog"}},"domain":{"domains":[{"id":"development","name":"development"},{"id":"staging","name":"staging"},{"id":"production","name":"production"}]},"enabled_plugins":{"tasks":{"task-plugins":{"default-for-task-types":{"container":"container","container_array":"k8s-array","sensor":"agent-service","sidecar":"sidecar"},"enabled-plugins":["container","sidecar","k8s-array","agent-service","echo"]}}},"k8s":{"plugins":{"k8s":{"default-cpus":"100m","default-env-from-configmaps":[],"default-env-from-secrets":[],"default-env-vars":[{"FLYTE_AWS_ENDPOINT":"http://minio.flyte:9000"},{"FLYTE_AWS_ACCESS_KEY_ID":"minio"},{"FLYTE_AWS_SECRET_ACCESS_KEY":"miniostorage"}],"default-memory":"200Mi"}}},"logger":{"logger":{"level":5,"show-source":true}},"remoteData":{"remoteData":{"region":"us-east-1","scheme":"local","signedUrls":{"durationMinutes":3}}},"resource_manager":{"propeller":{"resourcemanager":{"redis":null,"type":"noop"}}},"task_logs":{"plugins":{"logs":{"cloudwatch-enabled":false,"kubernetes-enabled":true,"kubernetes-template-uri":"http://localhost:30082/#/log/{{ \"{{\" }} .namespace {{ \"}}\" }}/{{ \"{{\" }} .podName {{ \"}}\" }}/pod?namespace={{ \"{{\" }} .namespace {{ \"}}\" }}"}}},"task_resource_defaults":{"task_resources":{"defaults":{"cpu":"100m","memory":"200Mi","storage":"5Mi"},"limits":{"cpu":2,"gpu":1,"memory":"1Gi","storage":"20Mi"}}}},"datacatalog":{"affinity":{},"configPath":"/etc/datacatalog/config/*.yaml","image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/datacatalog","tag":"v1.13.1"},"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"500m","ephemeral-storage":"100Mi","memory":"500Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"service":{"annotations":{"projectcontour.io/upstream-protocol.h2c":"grpc"},"type":"NodePort"},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"db":{"admin":{"database":{"dbname":"flyteadmin","host":"postgres","port":5432,"username":"postgres"}},"datacatalog":{"database":{"dbname":"datacatalog","host":"postgres","port":5432,"username":"postgres"}}},"deployRedoc":true,"flyteadmin":{"additionalVolumeMounts":[],"additionalVolumes":[],"affinity":{},"configPath":"/etc/flyte/config/*.yaml","env":[],"image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flyteadmin","tag":"v1.13.1"},"initialProjects":["flytesnacks","flytetester","flyteexamples"],"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"250m","ephemeral-storage":"100Mi","memory":"500Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"secrets":{},"service":{"annotations":{"projectcontour.io/upstream-protocol.h2c":"grpc"},"loadBalancerSourceRanges":[],"type":"ClusterIP"},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"flyteconsole":{"affinity":{},"ga":{"enabled":true,"tracking_id":"G-0QW4DJWJ20"},"image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flyteconsole","tag":"v1.17.1"},"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"500m","memory":"275Mi"},"requests":{"cpu":"10m","memory":"250Mi"}},"service":{"annotations":{},"type":"ClusterIP"},"tolerations":[]},"flytepropeller":{"affinity":{},"configPath":"/etc/flyte/config/*.yaml","image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flytepropeller","tag":"v1.13.1"},"manager":false,"nodeSelector":{},"podAnnotations":{},"replicaCount":1,"resources":{"limits":{"cpu":"200m","ephemeral-storage":"100Mi","memory":"200Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"flytescheduler":{"affinity":{},"configPath":"/etc/flyte/config/*.yaml","image":{"pullPolicy":"IfNotPresent","repository":"cr.flyte.org/flyteorg/flytescheduler","tag":"v1.13.1"},"nodeSelector":{},"podAnnotations":{},"resources":{"limits":{"cpu":"250m","ephemeral-storage":"100Mi","memory":"500Mi"},"requests":{"cpu":"10m","ephemeral-storage":"50Mi","memory":"50Mi"}},"secrets":{},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]},"tolerations":[]},"storage":{"bucketName":"my-s3-bucket","cache":{"maxSizeMBs":0,"targetGCPercent":70},"custom":{},"gcs":null,"s3":{"region":"us-east-1"},"type":"sandbox"},"webhook":{"enabled":true,"service":{"annotations":{"projectcontour.io/upstream-protocol.h2c":"grpc"},"type":"ClusterIP"},"serviceAccount":{"annotations":{},"create":true,"imagePullSecrets":[]}},"workflow_notifications":{"config":{},"enabled":false},"workflow_scheduler":{"enabled":true,"type":"native"}}` | ------------------------------------------------------------------- Core System settings This section consists of Core components of Flyte and their deployment settings. This includes FlyteAdmin service, Datacatalog, FlytePropeller and Flyteconsole | | flyte.cluster_resource_manager | object | `{"config":{"cluster_resources":{"customData":[{"production":[{"projectQuotaCpu":{"value":"5"}},{"projectQuotaMemory":{"value":"4000Mi"}}]},{"staging":[{"projectQuotaCpu":{"value":"2"}},{"projectQuotaMemory":{"value":"3000Mi"}}]},{"development":[{"projectQuotaCpu":{"value":"4"}},{"projectQuotaMemory":{"value":"3000Mi"}}]}],"refresh":"5m","refreshInterval":"5m","standaloneDeployment":false,"templatePath":"/etc/flyte/clusterresource/templates"}},"enabled":true,"service_account_name":"flyteadmin","templates":[{"key":"aa_namespace","value":"apiVersion: v1\nkind: Namespace\nmetadata:\n name: {{ namespace }}\nspec:\n finalizers:\n - kubernetes\n"},{"key":"ab_project_resource_quota","value":"apiVersion: v1\nkind: ResourceQuota\nmetadata:\n name: project-quota\n namespace: {{ namespace }}\nspec:\n hard:\n limits.cpu: {{ projectQuotaCpu }}\n limits.memory: {{ projectQuotaMemory }}\n"}]}` | Configuration for the Cluster resource manager component. This is an optional component, that enables automatic cluster configuration. This is useful to set default quotas, manage namespaces etc that map to a project/domain | | flyte.cluster_resource_manager.config.cluster_resources | object | `{"customData":[{"production":[{"projectQuotaCpu":{"value":"5"}},{"projectQuotaMemory":{"value":"4000Mi"}}]},{"staging":[{"projectQuotaCpu":{"value":"2"}},{"projectQuotaMemory":{"value":"3000Mi"}}]},{"development":[{"projectQuotaCpu":{"value":"4"}},{"projectQuotaMemory":{"value":"3000Mi"}}]}],"refresh":"5m","refreshInterval":"5m","standaloneDeployment":false,"templatePath":"/etc/flyte/clusterresource/templates"}` | ClusterResource parameters Refer to the [structure](https://pkg.go.dev/github.com/lyft/flyteadmin@v0.3.37/pkg/runtime/interfaces#ClusterResourceConfig) to customize. | | flyte.cluster_resource_manager.config.cluster_resources.standaloneDeployment | bool | `false` | Starts the cluster resource manager in standalone mode with requisite auth credentials to call flyteadmin service endpoints | @@ -185,7 +185,7 @@ helm upgrade -f values-sandbox.yaml flyte . | flyte.flytescheduler.serviceAccount.create | bool | `true` | Should a service account be created for Flytescheduler | | flyte.flytescheduler.serviceAccount.imagePullSecrets | list | `[]` | ImagePullSecrets to automatically assign to the service account | | flyte.flytescheduler.tolerations | list | `[]` | tolerations for Flytescheduler deployment | -| flyte.storage | object | `{"bucketName":"my-s3-bucket","custom":{},"gcs":null,"s3":{"region":"us-east-1"},"type":"sandbox"}` | ---------------------------------------------------- STORAGE SETTINGS | +| flyte.storage | object | `{"bucketName":"my-s3-bucket","cache":{"maxSizeMBs":0,"targetGCPercent":70},"custom":{},"gcs":null,"s3":{"region":"us-east-1"},"type":"sandbox"}` | ---------------------------------------------------- STORAGE SETTINGS | | flyte.storage.bucketName | string | `"my-s3-bucket"` | bucketName defines the storage bucket flyte will use. Required for all types except for sandbox. | | flyte.storage.custom | object | `{}` | Settings for storage type custom. See https://github.com/graymeta/stow for supported storage providers/settings. | | flyte.storage.gcs | string | `nil` | settings for storage type gcs | diff --git a/charts/flyte/values.yaml b/charts/flyte/values.yaml index 63c0a34c64..66144c7395 100755 --- a/charts/flyte/values.yaml +++ b/charts/flyte/values.yaml @@ -190,7 +190,6 @@ flyte: cpu: 10m ephemeral-storage: 50Mi memory: 50Mi - cacheSizeMbs: 0 # -- Default regex string for searching configuration files configPath: /etc/flyte/config/*.yaml @@ -333,6 +332,9 @@ flyte: # serviceAccountKey: "" # -- Settings for storage type custom. See https://github.com/graymeta/stow for supported storage providers/settings. custom: {} + cache: + maxSizeMBs: 0 + targetGCPercent: 70 # Database configuration db: diff --git a/deployment/eks/flyte_aws_scheduler_helm_generated.yaml b/deployment/eks/flyte_aws_scheduler_helm_generated.yaml index 9ee5eecc49..0d1e0b95b0 100644 --- a/deployment/eks/flyte_aws_scheduler_helm_generated.yaml +++ b/deployment/eks/flyte_aws_scheduler_helm_generated.yaml @@ -189,6 +189,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 1024 + target_gc_percent: 70 task_resource_defaults.yaml: | task_resources: defaults: @@ -398,6 +401,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 1024 + target_gc_percent: 70 --- # Source: flyte-core/templates/propeller/configmap.yaml apiVersion: v1 @@ -512,8 +518,6 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 - cache.yaml: | - storage: cache: max_size_mbs: 1024 target_gc_percent: 70 @@ -857,7 +861,7 @@ spec: template: metadata: annotations: - configChecksum: "618a516ca42e8bbe5222a76f7865a0a444b6048002d7fcc06144c9188f3fd3d" + configChecksum: "c943b200cd0bed97fe456c0c713dd79cdc4e22133495cac89db3fc55e9b79c7" labels: app.kubernetes.io/name: flyteadmin app.kubernetes.io/instance: flyte @@ -1177,7 +1181,7 @@ spec: template: metadata: annotations: - configChecksum: "c2a15ce5dc2fa465986d6006f93450723da58166b3ad5ee35a91cb37d5c39da" + configChecksum: "ded28f3a68d22eb8e5af14a44cc0d14326f10060405268aac5a3665fb86c8bc" labels: app.kubernetes.io/name: datacatalog app.kubernetes.io/instance: flyte @@ -1279,7 +1283,7 @@ spec: template: metadata: annotations: - configChecksum: "16400bbf28ab823ca433a04379f00fecb11530c3c265dd1bdc3e30209ac6a6d" + configChecksum: "79ddcbcba1c70aac67b013b43ab2068b8c0733b23a48da5b721162184ac5a02" prometheus.io/path: "/metrics" prometheus.io/port: "10254" labels: @@ -1363,7 +1367,7 @@ spec: app.kubernetes.io/name: flyte-pod-webhook app.kubernetes.io/version: v1.13.1 annotations: - configChecksum: "16400bbf28ab823ca433a04379f00fecb11530c3c265dd1bdc3e30209ac6a6d" + configChecksum: "79ddcbcba1c70aac67b013b43ab2068b8c0733b23a48da5b721162184ac5a02" prometheus.io/path: "/metrics" prometheus.io/port: "10254" spec: diff --git a/deployment/eks/flyte_helm_controlplane_generated.yaml b/deployment/eks/flyte_helm_controlplane_generated.yaml index ddc2116a6d..efb3da6611 100644 --- a/deployment/eks/flyte_helm_controlplane_generated.yaml +++ b/deployment/eks/flyte_helm_controlplane_generated.yaml @@ -170,6 +170,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 1024 + target_gc_percent: 70 task_resource_defaults.yaml: | task_resources: defaults: @@ -364,6 +367,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 1024 + target_gc_percent: 70 --- # Source: flyte-core/templates/flytescheduler/configmap.yaml apiVersion: v1 @@ -560,7 +566,7 @@ spec: template: metadata: annotations: - configChecksum: "5ce6f593fb92c9a6fd183825231d187471b5f10fe948f601f6d5b56edd02b51" + configChecksum: "391e8e126d669f751ac1a03de0b45fe7969a0fe58f3dfead9bb7be1b5d951ff" labels: app.kubernetes.io/name: flyteadmin app.kubernetes.io/instance: flyte @@ -880,7 +886,7 @@ spec: template: metadata: annotations: - configChecksum: "c2a15ce5dc2fa465986d6006f93450723da58166b3ad5ee35a91cb37d5c39da" + configChecksum: "ded28f3a68d22eb8e5af14a44cc0d14326f10060405268aac5a3665fb86c8bc" labels: app.kubernetes.io/name: datacatalog app.kubernetes.io/instance: flyte @@ -982,7 +988,7 @@ spec: template: metadata: annotations: - configChecksum: "5ce6f593fb92c9a6fd183825231d187471b5f10fe948f601f6d5b56edd02b51" + configChecksum: "391e8e126d669f751ac1a03de0b45fe7969a0fe58f3dfead9bb7be1b5d951ff" labels: app.kubernetes.io/name: flytescheduler app.kubernetes.io/instance: flyte diff --git a/deployment/eks/flyte_helm_dataplane_generated.yaml b/deployment/eks/flyte_helm_dataplane_generated.yaml index aa649b24ea..88708da79f 100644 --- a/deployment/eks/flyte_helm_dataplane_generated.yaml +++ b/deployment/eks/flyte_helm_dataplane_generated.yaml @@ -176,8 +176,6 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 - cache.yaml: | - storage: cache: max_size_mbs: 1024 target_gc_percent: 70 @@ -430,7 +428,7 @@ spec: template: metadata: annotations: - configChecksum: "16400bbf28ab823ca433a04379f00fecb11530c3c265dd1bdc3e30209ac6a6d" + configChecksum: "79ddcbcba1c70aac67b013b43ab2068b8c0733b23a48da5b721162184ac5a02" prometheus.io/path: "/metrics" prometheus.io/port: "10254" labels: @@ -514,7 +512,7 @@ spec: app.kubernetes.io/name: flyte-pod-webhook app.kubernetes.io/version: v1.13.1 annotations: - configChecksum: "16400bbf28ab823ca433a04379f00fecb11530c3c265dd1bdc3e30209ac6a6d" + configChecksum: "79ddcbcba1c70aac67b013b43ab2068b8c0733b23a48da5b721162184ac5a02" prometheus.io/path: "/metrics" prometheus.io/port: "10254" spec: diff --git a/deployment/eks/flyte_helm_generated.yaml b/deployment/eks/flyte_helm_generated.yaml index 6e95514c44..6f9bb5ecc0 100644 --- a/deployment/eks/flyte_helm_generated.yaml +++ b/deployment/eks/flyte_helm_generated.yaml @@ -201,6 +201,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 1024 + target_gc_percent: 70 task_resource_defaults.yaml: | task_resources: defaults: @@ -395,6 +398,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 1024 + target_gc_percent: 70 --- # Source: flyte-core/templates/flytescheduler/configmap.yaml apiVersion: v1 @@ -543,8 +549,6 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 - cache.yaml: | - storage: cache: max_size_mbs: 1024 target_gc_percent: 70 @@ -888,7 +892,7 @@ spec: template: metadata: annotations: - configChecksum: "5ce6f593fb92c9a6fd183825231d187471b5f10fe948f601f6d5b56edd02b51" + configChecksum: "391e8e126d669f751ac1a03de0b45fe7969a0fe58f3dfead9bb7be1b5d951ff" labels: app.kubernetes.io/name: flyteadmin app.kubernetes.io/instance: flyte @@ -1208,7 +1212,7 @@ spec: template: metadata: annotations: - configChecksum: "c2a15ce5dc2fa465986d6006f93450723da58166b3ad5ee35a91cb37d5c39da" + configChecksum: "ded28f3a68d22eb8e5af14a44cc0d14326f10060405268aac5a3665fb86c8bc" labels: app.kubernetes.io/name: datacatalog app.kubernetes.io/instance: flyte @@ -1310,7 +1314,7 @@ spec: template: metadata: annotations: - configChecksum: "5ce6f593fb92c9a6fd183825231d187471b5f10fe948f601f6d5b56edd02b51" + configChecksum: "391e8e126d669f751ac1a03de0b45fe7969a0fe58f3dfead9bb7be1b5d951ff" labels: app.kubernetes.io/name: flytescheduler app.kubernetes.io/instance: flyte @@ -1409,7 +1413,7 @@ spec: template: metadata: annotations: - configChecksum: "16400bbf28ab823ca433a04379f00fecb11530c3c265dd1bdc3e30209ac6a6d" + configChecksum: "79ddcbcba1c70aac67b013b43ab2068b8c0733b23a48da5b721162184ac5a02" prometheus.io/path: "/metrics" prometheus.io/port: "10254" labels: @@ -1493,7 +1497,7 @@ spec: app.kubernetes.io/name: flyte-pod-webhook app.kubernetes.io/version: v1.13.1 annotations: - configChecksum: "16400bbf28ab823ca433a04379f00fecb11530c3c265dd1bdc3e30209ac6a6d" + configChecksum: "79ddcbcba1c70aac67b013b43ab2068b8c0733b23a48da5b721162184ac5a02" prometheus.io/path: "/metrics" prometheus.io/port: "10254" spec: diff --git a/deployment/gcp/flyte_helm_controlplane_generated.yaml b/deployment/gcp/flyte_helm_controlplane_generated.yaml index 43245474e9..be8594fc29 100644 --- a/deployment/gcp/flyte_helm_controlplane_generated.yaml +++ b/deployment/gcp/flyte_helm_controlplane_generated.yaml @@ -175,6 +175,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 0 + target_gc_percent: 70 task_resource_defaults.yaml: | task_resources: defaults: @@ -377,6 +380,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 0 + target_gc_percent: 70 --- # Source: flyte-core/templates/flytescheduler/configmap.yaml apiVersion: v1 @@ -575,7 +581,7 @@ spec: template: metadata: annotations: - configChecksum: "0705f122f2535babec96a6083827c3e6d27e6e9b0e460b4d07292c858079ac7" + configChecksum: "20a517901c6b6f01f47e968fa15ca51f6d9522e728ecace8b48553eb428cde6" labels: app.kubernetes.io/name: flyteadmin app.kubernetes.io/instance: flyte @@ -895,7 +901,7 @@ spec: template: metadata: annotations: - configChecksum: "905a2a911a85dbf8d4f1dc3be24b9c4fd7bb46481db0e174274d6aea6129b4c" + configChecksum: "537b12b49584e5eb9da85bbf0d8d8d21d8edce0560b0b53f595485f2cdb1cb6" labels: app.kubernetes.io/name: datacatalog app.kubernetes.io/instance: flyte @@ -997,7 +1003,7 @@ spec: template: metadata: annotations: - configChecksum: "0705f122f2535babec96a6083827c3e6d27e6e9b0e460b4d07292c858079ac7" + configChecksum: "20a517901c6b6f01f47e968fa15ca51f6d9522e728ecace8b48553eb428cde6" labels: app.kubernetes.io/name: flytescheduler app.kubernetes.io/instance: flyte diff --git a/deployment/gcp/flyte_helm_dataplane_generated.yaml b/deployment/gcp/flyte_helm_dataplane_generated.yaml index 40f2542640..79cb10d579 100644 --- a/deployment/gcp/flyte_helm_dataplane_generated.yaml +++ b/deployment/gcp/flyte_helm_dataplane_generated.yaml @@ -179,10 +179,8 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 - cache.yaml: | - storage: cache: - max_size_mbs: 1024 + max_size_mbs: 0 target_gc_percent: 70 task_logs.yaml: | plugins: @@ -438,7 +436,7 @@ spec: template: metadata: annotations: - configChecksum: "b11d6c5dd0dd16bced82fb44fd2fb31c2f27a134543ab7edaf060da42ab27e4" + configChecksum: "d2d16163f314c9a2ffa243dd08687e8040515158b29593aea690ac97c10f343" prometheus.io/path: "/metrics" prometheus.io/port: "10254" labels: @@ -521,7 +519,7 @@ spec: app.kubernetes.io/name: flyte-pod-webhook app.kubernetes.io/version: v1.13.1 annotations: - configChecksum: "b11d6c5dd0dd16bced82fb44fd2fb31c2f27a134543ab7edaf060da42ab27e4" + configChecksum: "d2d16163f314c9a2ffa243dd08687e8040515158b29593aea690ac97c10f343" prometheus.io/path: "/metrics" prometheus.io/port: "10254" spec: diff --git a/deployment/gcp/flyte_helm_generated.yaml b/deployment/gcp/flyte_helm_generated.yaml index 6f75ece6a1..3585027b15 100644 --- a/deployment/gcp/flyte_helm_generated.yaml +++ b/deployment/gcp/flyte_helm_generated.yaml @@ -206,6 +206,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 0 + target_gc_percent: 70 task_resource_defaults.yaml: | task_resources: defaults: @@ -408,6 +411,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 0 + target_gc_percent: 70 --- # Source: flyte-core/templates/flytescheduler/configmap.yaml apiVersion: v1 @@ -559,10 +565,8 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 - cache.yaml: | - storage: cache: - max_size_mbs: 1024 + max_size_mbs: 0 target_gc_percent: 70 task_logs.yaml: | plugins: @@ -911,7 +915,7 @@ spec: template: metadata: annotations: - configChecksum: "0705f122f2535babec96a6083827c3e6d27e6e9b0e460b4d07292c858079ac7" + configChecksum: "20a517901c6b6f01f47e968fa15ca51f6d9522e728ecace8b48553eb428cde6" labels: app.kubernetes.io/name: flyteadmin app.kubernetes.io/instance: flyte @@ -1231,7 +1235,7 @@ spec: template: metadata: annotations: - configChecksum: "905a2a911a85dbf8d4f1dc3be24b9c4fd7bb46481db0e174274d6aea6129b4c" + configChecksum: "537b12b49584e5eb9da85bbf0d8d8d21d8edce0560b0b53f595485f2cdb1cb6" labels: app.kubernetes.io/name: datacatalog app.kubernetes.io/instance: flyte @@ -1333,7 +1337,7 @@ spec: template: metadata: annotations: - configChecksum: "0705f122f2535babec96a6083827c3e6d27e6e9b0e460b4d07292c858079ac7" + configChecksum: "20a517901c6b6f01f47e968fa15ca51f6d9522e728ecace8b48553eb428cde6" labels: app.kubernetes.io/name: flytescheduler app.kubernetes.io/instance: flyte @@ -1432,7 +1436,7 @@ spec: template: metadata: annotations: - configChecksum: "b11d6c5dd0dd16bced82fb44fd2fb31c2f27a134543ab7edaf060da42ab27e4" + configChecksum: "d2d16163f314c9a2ffa243dd08687e8040515158b29593aea690ac97c10f343" prometheus.io/path: "/metrics" prometheus.io/port: "10254" labels: @@ -1515,7 +1519,7 @@ spec: app.kubernetes.io/name: flyte-pod-webhook app.kubernetes.io/version: v1.13.1 annotations: - configChecksum: "b11d6c5dd0dd16bced82fb44fd2fb31c2f27a134543ab7edaf060da42ab27e4" + configChecksum: "d2d16163f314c9a2ffa243dd08687e8040515158b29593aea690ac97c10f343" prometheus.io/path: "/metrics" prometheus.io/port: "10254" spec: diff --git a/deployment/sandbox/flyte_helm_generated.yaml b/deployment/sandbox/flyte_helm_generated.yaml index 9072c486d3..f580d42de7 100644 --- a/deployment/sandbox/flyte_helm_generated.yaml +++ b/deployment/sandbox/flyte_helm_generated.yaml @@ -331,6 +331,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 0 + target_gc_percent: 70 task_resource_defaults.yaml: | task_resources: defaults: @@ -517,6 +520,9 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 + cache: + max_size_mbs: 0 + target_gc_percent: 70 --- # Source: flyte/charts/flyte/templates/flytescheduler/configmap.yaml apiVersion: v1 @@ -682,8 +688,6 @@ data: enable-multicontainer: false limits: maxDownloadMBs: 10 - cache.yaml: | - storage: cache: max_size_mbs: 0 target_gc_percent: 70 @@ -6695,7 +6699,7 @@ spec: template: metadata: annotations: - configChecksum: "88625d852360c42642190e21751f32f7dd9501ce3d479fa68b86478995ff689" + configChecksum: "f2d2bbea27b58cc5a73da30eb8aeb56fc41863f4eba2bfe407da2e97a6372e8" labels: app.kubernetes.io/name: flyteadmin app.kubernetes.io/instance: flyte @@ -6997,7 +7001,7 @@ spec: template: metadata: annotations: - configChecksum: "ccdd0d27618b8053a8ae11046fd2b84b9a397144dd81c7113f398cddf001397" + configChecksum: "0df67e720160bb897fcb950f39eede7efbd668c770872d171469df8dc1dd70f" labels: app.kubernetes.io/name: datacatalog app.kubernetes.io/instance: flyte @@ -7088,7 +7092,7 @@ spec: template: metadata: annotations: - configChecksum: "88625d852360c42642190e21751f32f7dd9501ce3d479fa68b86478995ff689" + configChecksum: "f2d2bbea27b58cc5a73da30eb8aeb56fc41863f4eba2bfe407da2e97a6372e8" labels: app.kubernetes.io/name: flytescheduler app.kubernetes.io/instance: flyte @@ -7183,7 +7187,7 @@ spec: template: metadata: annotations: - configChecksum: "8a003fdbed4b3801328c26cb5c202e4ca113875347388366d29a302d610e7e4" + configChecksum: "84d938f8e97d5cbfa3863167274019cdfd36e1fa78b623a49e963796a760ec7" prometheus.io/path: "/metrics" prometheus.io/port: "10254" labels: @@ -7259,7 +7263,7 @@ spec: app.kubernetes.io/name: flyte-pod-webhook app.kubernetes.io/version: v1.13.1 annotations: - configChecksum: "8a003fdbed4b3801328c26cb5c202e4ca113875347388366d29a302d610e7e4" + configChecksum: "84d938f8e97d5cbfa3863167274019cdfd36e1fa78b623a49e963796a760ec7" prometheus.io/path: "/metrics" prometheus.io/port: "10254" spec: diff --git a/docker/sandbox-bundled/manifests/complete-agent.yaml b/docker/sandbox-bundled/manifests/complete-agent.yaml index ea6321200d..4e2c5a446d 100644 --- a/docker/sandbox-bundled/manifests/complete-agent.yaml +++ b/docker/sandbox-bundled/manifests/complete-agent.yaml @@ -816,7 +816,7 @@ type: Opaque --- apiVersion: v1 data: - haSharedSecret: ZXlJVkhWYjdIMHhjamZadA== + haSharedSecret: WXpIVGJBbE9kYUFUZXB4NQ== proxyPassword: "" proxyUsername: "" kind: Secret @@ -1413,7 +1413,7 @@ spec: metadata: annotations: checksum/config: 8f50e768255a87f078ba8b9879a0c174c3e045ffb46ac8723d2eedbe293c8d81 - checksum/secret: 94a4c448ea7ad0892283bc4cfc6c506c83c9c5fe998587f4b2c55194c6a674e3 + checksum/secret: 749d365b51bf3ab24a58718117258b5ab7458e62a369ed68f1bc6bcc5c0af298 labels: app: docker-registry release: flyte-sandbox diff --git a/docker/sandbox-bundled/manifests/complete.yaml b/docker/sandbox-bundled/manifests/complete.yaml index 3437469a1c..a5b86c2456 100644 --- a/docker/sandbox-bundled/manifests/complete.yaml +++ b/docker/sandbox-bundled/manifests/complete.yaml @@ -798,7 +798,7 @@ type: Opaque --- apiVersion: v1 data: - haSharedSecret: OW1PbDdRY0t4RllhM3Nybg== + haSharedSecret: YXQ2OWxIVXFIMWZVanlTaw== proxyPassword: "" proxyUsername: "" kind: Secret @@ -1362,7 +1362,7 @@ spec: metadata: annotations: checksum/config: 8f50e768255a87f078ba8b9879a0c174c3e045ffb46ac8723d2eedbe293c8d81 - checksum/secret: 1f30487909a5b2db21b8f92a734fcb321ab30f01694f4257333026e00d512053 + checksum/secret: bc5ce6c562bb26ef159fcd62f9dea0d7673290742af0e6f2870d76593c00a3c2 labels: app: docker-registry release: flyte-sandbox diff --git a/docker/sandbox-bundled/manifests/dev.yaml b/docker/sandbox-bundled/manifests/dev.yaml index f0e2a866af..29b385b467 100644 --- a/docker/sandbox-bundled/manifests/dev.yaml +++ b/docker/sandbox-bundled/manifests/dev.yaml @@ -499,7 +499,7 @@ metadata: --- apiVersion: v1 data: - haSharedSecret: MWVqaUwzWDZtUWY4TDdscA== + haSharedSecret: NUw2M1B4VHZQbUJLV01rag== proxyPassword: "" proxyUsername: "" kind: Secret @@ -934,7 +934,7 @@ spec: metadata: annotations: checksum/config: 8f50e768255a87f078ba8b9879a0c174c3e045ffb46ac8723d2eedbe293c8d81 - checksum/secret: 53219c6f309435a180b4635448e130a2ec19b63b379a881dde73bf8ae957a1ad + checksum/secret: 170f57b7df390c2e687f7843ef51d301344f2baeff1ccca736118deaadf3dda5 labels: app: docker-registry release: flyte-sandbox From 69f47dd1088e782618a8a4310e3c39522737d376 Mon Sep 17 00:00:00 2001 From: Dan Rammer Date: Mon, 8 Apr 2024 13:50:59 -0500 Subject: [PATCH 12/25] added lock to memstore make threadsafe Signed-off-by: Andrew Dye --- flytestdlib/storage/mem_store.go | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/flytestdlib/storage/mem_store.go b/flytestdlib/storage/mem_store.go index 94083f6646..d9da9b5b1e 100644 --- a/flytestdlib/storage/mem_store.go +++ b/flytestdlib/storage/mem_store.go @@ -9,13 +9,15 @@ import ( "io" "io/ioutil" "os" + "sync" ) type rawFile = []byte type InMemoryStore struct { copyImpl - cache map[DataReference]rawFile + cache map[DataReference]rawFile + rwMutex sync.RWMutex } type MemoryMetadata struct { @@ -42,6 +44,9 @@ func (m MemoryMetadata) ContentMD5() string { } func (s *InMemoryStore) Head(ctx context.Context, reference DataReference) (Metadata, error) { + s.rwMutex.RLock() + defer s.rwMutex.RUnlock() + data, found := s.cache[reference] var hash [md5.Size]byte if found { @@ -59,6 +64,9 @@ func (s *InMemoryStore) List(ctx context.Context, reference DataReference, maxIt } func (s *InMemoryStore) ReadRaw(ctx context.Context, reference DataReference) (io.ReadCloser, error) { + s.rwMutex.RLock() + defer s.rwMutex.RUnlock() + if raw, found := s.cache[reference]; found { return ioutil.NopCloser(bytes.NewReader(raw)), nil } @@ -68,6 +76,9 @@ func (s *InMemoryStore) ReadRaw(ctx context.Context, reference DataReference) (i // Delete removes the referenced data from the cache map. func (s *InMemoryStore) Delete(ctx context.Context, reference DataReference) error { + s.rwMutex.Lock() + defer s.rwMutex.Unlock() + if _, found := s.cache[reference]; !found { return os.ErrNotExist } @@ -79,6 +90,8 @@ func (s *InMemoryStore) Delete(ctx context.Context, reference DataReference) err func (s *InMemoryStore) WriteRaw(ctx context.Context, reference DataReference, size int64, opts Options, raw io.Reader) ( err error) { + s.rwMutex.Lock() + defer s.rwMutex.Unlock() rawBytes, err := ioutil.ReadAll(raw) if err != nil { @@ -90,6 +103,9 @@ func (s *InMemoryStore) WriteRaw(ctx context.Context, reference DataReference, s } func (s *InMemoryStore) Clear(ctx context.Context) error { + s.rwMutex.Lock() + defer s.rwMutex.Unlock() + s.cache = map[DataReference]rawFile{} return nil } From 071c137afb578661ee5bc84b768391bd7176b6f2 Mon Sep 17 00:00:00 2001 From: Joe Eschen <126913098+squiishyy@users.noreply.github.com> Date: Wed, 24 Apr 2024 15:27:45 -0700 Subject: [PATCH 13/25] Add read replica host config and connection - Add a new field to the postgres db config struct, `readReplicaHost`. - Add a new endpoint in the `database` package to enable establishing a connection with a db without creating it if it doesn't exist Signed-off-by: Andrew Dye --- flytestdlib/database/config.go | 22 ++++++----- flytestdlib/database/db.go | 25 +++++++++++++ flytestdlib/database/dbconfig_flags.go | 1 + flytestdlib/database/dbconfig_flags_test.go | 14 +++++++ flytestdlib/database/postgres.go | 18 +++++++++ flytestdlib/database/postgres_test.go | 41 +++++++++++++++++++++ 6 files changed, 111 insertions(+), 10 deletions(-) diff --git a/flytestdlib/database/config.go b/flytestdlib/database/config.go index 16ca0e5708..f55eecda8f 100644 --- a/flytestdlib/database/config.go +++ b/flytestdlib/database/config.go @@ -19,12 +19,13 @@ var defaultConfig = &DbConfig{ ConnMaxLifeTime: config.Duration{Duration: time.Hour}, Postgres: PostgresConfig{ // These values are suitable for local sandbox development - Host: "localhost", - Port: 30001, - DbName: postgresStr, - User: postgresStr, - Password: postgresStr, - ExtraOptions: "sslmode=disable", + Host: "localhost", + ReadReplicaHost: "localhost", + Port: 30001, + DbName: postgresStr, + User: postgresStr, + Password: postgresStr, + ExtraOptions: "sslmode=disable", }, } var configSection = config.MustRegisterSection(database, defaultConfig) @@ -64,10 +65,11 @@ type SQLiteConfig struct { // PostgresConfig includes specific config options for opening a connection to a postgres database. type PostgresConfig struct { - Host string `json:"host" pflag:",The host name of the database server"` - Port int `json:"port" pflag:",The port name of the database server"` - DbName string `json:"dbname" pflag:",The database name"` - User string `json:"username" pflag:",The database user who is connecting to the server."` + Host string `json:"host" pflag:",The host name of the database server"` + ReadReplicaHost string `json:"readReplicaHost" pflag:",The host name of the read replica database server"` + Port int `json:"port" pflag:",The port name of the database server"` + DbName string `json:"dbname" pflag:",The database name"` + User string `json:"username" pflag:",The database user who is connecting to the server."` // Either Password or PasswordPath must be set. Password string `json:"password" pflag:",The database password."` PasswordPath string `json:"passwordPath" pflag:",Points to the file containing the database password."` diff --git a/flytestdlib/database/db.go b/flytestdlib/database/db.go index 8046c7ead4..a964518567 100644 --- a/flytestdlib/database/db.go +++ b/flytestdlib/database/db.go @@ -65,6 +65,31 @@ func GetDB(ctx context.Context, dbConfig *DbConfig, logConfig *logger.Config) ( return gormDb, setupDbConnectionPool(ctx, gormDb, dbConfig) } +// GetReadOnlyDB uses the dbConfig to create gorm DB object for the read replica passed via the config +func GetReadOnlyDB(ctx context.Context, dbConfig *DbConfig, logConfig *logger.Config) (*gorm.DB, error) { + if dbConfig == nil { + panic("Cannot initialize database repository from empty db config") + } + + if dbConfig.Postgres.IsEmpty() || dbConfig.Postgres.ReadReplicaHost == "" { + return nil, fmt.Errorf("read replica host not provided in db config") + } + + gormConfig := &gorm.Config{ + Logger: GetGormLogger(ctx, logConfig), + DisableForeignKeyConstraintWhenMigrating: false, + } + + var gormDb *gorm.DB + var err error + gormDb, err = CreatePostgresReadOnlyDbConnection(ctx, gormConfig, dbConfig.Postgres) + if err != nil { + return nil, err + } + + return gormDb, nil +} + func setupDbConnectionPool(ctx context.Context, gormDb *gorm.DB, dbConfig *DbConfig) error { genericDb, err := gormDb.DB() if err != nil { diff --git a/flytestdlib/database/dbconfig_flags.go b/flytestdlib/database/dbconfig_flags.go index c925094cc2..9fa96f9fa8 100755 --- a/flytestdlib/database/dbconfig_flags.go +++ b/flytestdlib/database/dbconfig_flags.go @@ -55,6 +55,7 @@ func (cfg DbConfig) GetPFlagSet(prefix string) *pflag.FlagSet { cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "maxOpenConnections"), defaultConfig.MaxOpenConnections, "maxOpenConnections sets the maximum number of open connections to the database.") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "connMaxLifeTime"), defaultConfig.ConnMaxLifeTime.String(), "sets the maximum amount of time a connection may be reused") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "postgres.host"), defaultConfig.Postgres.Host, "The host name of the database server") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "postgres.readReplicaHost"), defaultConfig.Postgres.ReadReplicaHost, "The host name of the read replica database server") cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "postgres.port"), defaultConfig.Postgres.Port, "The port name of the database server") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "postgres.dbname"), defaultConfig.Postgres.DbName, "The database name") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "postgres.username"), defaultConfig.Postgres.User, "The database user who is connecting to the server.") diff --git a/flytestdlib/database/dbconfig_flags_test.go b/flytestdlib/database/dbconfig_flags_test.go index 2f0a5d53eb..fd49e69fd8 100755 --- a/flytestdlib/database/dbconfig_flags_test.go +++ b/flytestdlib/database/dbconfig_flags_test.go @@ -169,6 +169,20 @@ func TestDbConfig_SetFlags(t *testing.T) { } }) }) + t.Run("Test_postgres.readReplicaHost", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("postgres.readReplicaHost", testValue) + if vString, err := cmdFlags.GetString("postgres.readReplicaHost"); err == nil { + testDecodeJson_DbConfig(t, fmt.Sprintf("%v", vString), &actual.Postgres.ReadReplicaHost) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) t.Run("Test_postgres.port", func(t *testing.T) { t.Run("Override", func(t *testing.T) { diff --git a/flytestdlib/database/postgres.go b/flytestdlib/database/postgres.go index ec43330af1..7e73226465 100644 --- a/flytestdlib/database/postgres.go +++ b/flytestdlib/database/postgres.go @@ -54,6 +54,18 @@ func getPostgresDsn(ctx context.Context, pgConfig PostgresConfig) string { pgConfig.Host, pgConfig.Port, pgConfig.DbName, pgConfig.User, password, pgConfig.ExtraOptions) } +// Produces the DSN (data source name) for the read replica for opening a postgres db connection. +func getPostgresReadDsn(ctx context.Context, pgConfig PostgresConfig) string { + password := resolvePassword(ctx, pgConfig.Password, pgConfig.PasswordPath) + if len(password) == 0 { + // The password-less case is included for development environments. + return fmt.Sprintf("host=%s port=%d dbname=%s user=%s sslmode=disable", + pgConfig.ReadReplicaHost, pgConfig.Port, pgConfig.DbName, pgConfig.User) + } + return fmt.Sprintf("host=%s port=%d dbname=%s user=%s password=%s %s", + pgConfig.ReadReplicaHost, pgConfig.Port, pgConfig.DbName, pgConfig.User, password, pgConfig.ExtraOptions) +} + // CreatePostgresDbIfNotExists creates DB if it doesn't exist for the passed in config func CreatePostgresDbIfNotExists(ctx context.Context, gormConfig *gorm.Config, pgConfig PostgresConfig) (*gorm.DB, error) { dialector := postgres.Open(getPostgresDsn(ctx, pgConfig)) @@ -94,6 +106,12 @@ func CreatePostgresDbIfNotExists(ctx context.Context, gormConfig *gorm.Config, p return gorm.Open(dialector, gormConfig) } +// CreatePostgresDbConnection creates DB connection and returns the gorm.DB object and error +func CreatePostgresReadOnlyDbConnection(ctx context.Context, gormConfig *gorm.Config, pgConfig PostgresConfig) (*gorm.DB, error) { + dialector := postgres.Open(getPostgresReadDsn(ctx, pgConfig)) + return gorm.Open(dialector, gormConfig) +} + func IsPgErrorWithCode(err error, code string) bool { // Newer versions of the gorm postgres driver seem to use // "github.com/jackc/pgx/v5/pgconn" diff --git a/flytestdlib/database/postgres_test.go b/flytestdlib/database/postgres_test.go index 311b05c351..eb22c6b3aa 100644 --- a/flytestdlib/database/postgres_test.go +++ b/flytestdlib/database/postgres_test.go @@ -66,6 +66,47 @@ func TestGetPostgresDsn(t *testing.T) { }) } +func TestGetPostgresReadDsn(t *testing.T) { + pgConfig := PostgresConfig{ + Host: "localhost", + ReadReplicaHost: "readReplicaHost", + Port: 5432, + DbName: "postgres", + User: "postgres", + ExtraOptions: "sslmode=disable", + } + t.Run("no password", func(t *testing.T) { + dsn := getPostgresReadDsn(context.TODO(), pgConfig) + assert.Equal(t, "host=readReplicaHost port=5432 dbname=postgres user=postgres sslmode=disable", dsn) + }) + t.Run("with password", func(t *testing.T) { + pgConfig.Password = "passw" + dsn := getPostgresReadDsn(context.TODO(), pgConfig) + assert.Equal(t, "host=readReplicaHost port=5432 dbname=postgres user=postgres password=passw sslmode=disable", dsn) + + }) + t.Run("with password, no extra", func(t *testing.T) { + pgConfig.Password = "passwo" + pgConfig.ExtraOptions = "" + dsn := getPostgresReadDsn(context.TODO(), pgConfig) + assert.Equal(t, "host=readReplicaHost port=5432 dbname=postgres user=postgres password=passwo ", dsn) + }) + t.Run("with password path", func(t *testing.T) { + password := "1234abc" + tmpFile, err := ioutil.TempFile("", "prefix") + if err != nil { + t.Errorf("Couldn't open temp file: %v", err) + } + defer tmpFile.Close() + if _, err = tmpFile.WriteString(password); err != nil { + t.Errorf("Couldn't write to temp file: %v", err) + } + pgConfig.PasswordPath = tmpFile.Name() + dsn := getPostgresReadDsn(context.TODO(), pgConfig) + assert.Equal(t, "host=readReplicaHost port=5432 dbname=postgres user=postgres password=1234abc ", dsn) + }) +} + type wrappedError struct { err error } From 05dfd9df8b63679a747270504103aceb7ffab858 Mon Sep 17 00:00:00 2001 From: Haytham Abuelfutuh Date: Thu, 25 Apr 2024 14:42:47 -0700 Subject: [PATCH 14/25] =?UTF-8?q?Fix=20type=20assertion=20when=20an=20even?= =?UTF-8?q?t=20is=20missed=20while=20connection=20to=20apiser=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …ver was severed Signed-off-by: Andrew Dye --- .../nodes/task/k8s/event_watcher.go | 14 ++++++++++++- .../nodes/task/k8s/event_watcher_test.go | 20 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher.go b/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher.go index e53de83e10..13ebbc4cc5 100644 --- a/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher.go +++ b/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher.go @@ -72,7 +72,19 @@ func (e *eventWatcher) OnUpdate(_, newObj interface{}) { } func (e *eventWatcher) OnDelete(obj interface{}) { - event := obj.(*eventsv1.Event) + event, casted := obj.(*eventsv1.Event) + if !casted { + unknown, casted := obj.(cache.DeletedFinalStateUnknown) + if !casted { + logger.Warnf(context.Background(), "Unknown object type [%T] in OnDelete", obj) + } else { + logger.Warnf(context.Background(), "Deleted object of unknown key [%v] type [%T] in OnDelete", + unknown.Key, unknown.Obj) + } + + return + } + objectNsName := types.NamespacedName{Namespace: event.Regarding.Namespace, Name: event.Regarding.Name} eventNsName := types.NamespacedName{Namespace: event.Namespace, Name: event.Name} v, _ := e.objectCache.LoadOrStore(objectNsName, &objectEvents{}) diff --git a/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher_test.go b/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher_test.go index d3ffbcc5b9..37e4ba11ff 100644 --- a/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher_test.go +++ b/flytepropeller/pkg/controller/nodes/task/k8s/event_watcher_test.go @@ -9,6 +9,7 @@ import ( eventsv1 "k8s.io/api/events/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/types" + "k8s.io/client-go/tools/cache" ) func TestEventWatcher_OnAdd(t *testing.T) { @@ -143,6 +144,25 @@ func TestEventWatcher_OnDelete(t *testing.T) { v, _ := ew.objectCache.Load(types.NamespacedName{Namespace: "ns3", Name: "name3"}) assert.Nil(t, v) }) + + t.Run("bad object type", func(t *testing.T) { + ew.OnDelete(cache.DeletedFinalStateUnknown{ + Key: "key", + Obj: &eventsv1.Event{ + ObjectMeta: metav1.ObjectMeta{ + Namespace: "eventns3", + Name: "eventname3", + }, + Regarding: corev1.ObjectReference{ + Namespace: "ns3", + Name: "name3", + }, + }, + }) + + v, _ := ew.objectCache.Load(types.NamespacedName{Namespace: "ns3", Name: "name3"}) + assert.Nil(t, v) + }) } func TestEventWatcher_List(t *testing.T) { From 7ce2ca8545354a985acd8889e316b6edf8a2a0f8 Mon Sep 17 00:00:00 2001 From: Katrina Rogan Date: Mon, 29 Apr 2024 18:00:54 -0700 Subject: [PATCH 15/25] Log and monitor failures to validate access tokens Signed-off-by: Andrew Dye --- flyteadmin/auth/authzserver/provider.go | 29 ++++++++++++++++-- flyteadmin/auth/authzserver/provider_test.go | 7 +++-- flyteadmin/auth/handlers.go | 12 ++++---- flyteadmin/pkg/server/service.go | 4 +-- .../clients/go/admin/token_source_provider.go | 30 ++++++++++++++++--- 5 files changed, 66 insertions(+), 16 deletions(-) diff --git a/flyteadmin/auth/authzserver/provider.go b/flyteadmin/auth/authzserver/provider.go index b2948331fb..7df738a2d8 100644 --- a/flyteadmin/auth/authzserver/provider.go +++ b/flyteadmin/auth/authzserver/provider.go @@ -16,6 +16,7 @@ import ( fositeOAuth2 "github.com/ory/fosite/handler/oauth2" "github.com/ory/fosite/storage" "github.com/ory/fosite/token/jwt" + "github.com/prometheus/client_golang/prometheus" "k8s.io/apimachinery/pkg/util/sets" "github.com/flyteorg/flyte/flyteadmin/auth" @@ -24,6 +25,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/service" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyte/flytestdlib/logger" + "github.com/flyteorg/flyte/flytestdlib/promutils" ) const ( @@ -33,12 +35,18 @@ const ( KeyIDClaim = "key_id" ) +type providerMetrics struct { + InvalidTokens prometheus.Counter + ExpiredTokens prometheus.Counter +} + // Provider implements OAuth2 Authorization Server. type Provider struct { fosite.OAuth2Provider cfg config.AuthorizationServer publicKey []rsa.PublicKey keySet jwk.Set + metrics providerMetrics } func (p Provider) PublicKeys() []rsa.PublicKey { @@ -111,15 +119,28 @@ func (p Provider) ValidateAccessToken(ctx context.Context, expectedAudience, tok }) if err != nil { + logger.Infof(ctx, "failed to parse token for audience '%s'. Error: %v", expectedAudience, err) return nil, err } if !parsedToken.Valid { + if ve, ok := err.(*jwtgo.ValidationError); ok && ve.Is(jwtgo.ErrTokenExpired) { + logger.Infof(ctx, "parsed token for audience '%s' is expired", expectedAudience) + p.metrics.ExpiredTokens.Inc() + } else { + logger.Infof(ctx, "parsed token for audience '%s' is invalid: %+v", expectedAudience, err) + p.metrics.InvalidTokens.Inc() + } return nil, fmt.Errorf("parsed token is invalid") } claimsRaw := parsedToken.Claims.(jwtgo.MapClaims) - return verifyClaims(sets.NewString(expectedAudience), claimsRaw) + identityCtx, err := verifyClaims(sets.NewString(expectedAudience), claimsRaw) + if err != nil { + logger.Infof(ctx, "failed to verify claims for audience: '%s'. Error: %v", expectedAudience, err) + return nil, err + } + return identityCtx, nil } // NewProvider creates a new OAuth2 Provider that is able to do OAuth 2-legged and 3-legged flows. It'll lookup @@ -127,7 +148,7 @@ func (p Provider) ValidateAccessToken(ctx context.Context, expectedAudience, tok // sign and generate hashes for tokens. The RSA Private key is expected to be in PEM format with the public key embedded. // Use auth.GetInitSecretsCommand() to generate new valid secrets that will be accepted by this provider. // The config.SecretNameClaimSymmetricKey must be a 32-bytes long key in Base64Encoding. -func NewProvider(ctx context.Context, cfg config.AuthorizationServer, sm core.SecretManager) (Provider, error) { +func NewProvider(ctx context.Context, cfg config.AuthorizationServer, sm core.SecretManager, scope promutils.Scope) (Provider, error) { // fosite requires four parameters for the server to get up and running: // 1. config - for any enforcement you may desire, you can do this using `compose.Config`. You like PKCE, enforce it! // 2. store - no auth service is generally useful unless it can remember clients and users. @@ -230,5 +251,9 @@ func NewProvider(ctx context.Context, cfg config.AuthorizationServer, sm core.Se OAuth2Provider: oauth2Provider, publicKey: publicKeys, keySet: keysSet, + metrics: providerMetrics{ + ExpiredTokens: scope.MustNewCounter("expired_token", "The number of expired tokens"), + InvalidTokens: scope.MustNewCounter("invalid_tokens", "The number of invalid tokens"), + }, }, nil } diff --git a/flyteadmin/auth/authzserver/provider_test.go b/flyteadmin/auth/authzserver/provider_test.go index 45f0778b51..50fcaa5de1 100644 --- a/flyteadmin/auth/authzserver/provider_test.go +++ b/flyteadmin/auth/authzserver/provider_test.go @@ -18,6 +18,7 @@ import ( "github.com/flyteorg/flyte/flyteadmin/auth/config" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/service" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core/mocks" + "github.com/flyteorg/flyte/flytestdlib/promutils" ) func newMockProvider(t testing.TB) (Provider, auth.SecretsSet) { @@ -36,7 +37,7 @@ func newMockProvider(t testing.TB) (Provider, auth.SecretsSet) { sm.OnGet(ctx, config.SecretNameTokenSigningRSAKey).Return(buf.String(), nil) sm.OnGet(ctx, config.SecretNameOldTokenSigningRSAKey).Return(buf.String(), nil) - p, err := NewProvider(ctx, config.DefaultConfig.AppAuth.SelfAuthServer, sm) + p, err := NewProvider(ctx, config.DefaultConfig.AppAuth.SelfAuthServer, sm, promutils.NewTestScope()) assert.NoError(t, err) return p, secrets } @@ -58,7 +59,7 @@ func newInvalidMockProvider(ctx context.Context, t *testing.T, secrets auth.Secr sm.OnGet(ctx, config.SecretNameOldTokenSigningRSAKey).Return(buf.String(), nil) invalidFunc() - p, err := NewProvider(ctx, config.DefaultConfig.AppAuth.SelfAuthServer, sm) + p, err := NewProvider(ctx, config.DefaultConfig.AppAuth.SelfAuthServer, sm, promutils.NewTestScope()) assert.Error(t, err) assert.ErrorContains(t, err, errorContains) assert.Equal(t, Provider{}, p) @@ -294,7 +295,7 @@ func TestProvider_ValidateAccessToken(t *testing.T) { sm.OnGet(ctx, config.SecretNameTokenSigningRSAKey).Return(buf.String(), nil) sm.OnGet(ctx, config.SecretNameOldTokenSigningRSAKey).Return(buf.String(), nil) - p, err := NewProvider(ctx, config.DefaultConfig.AppAuth.SelfAuthServer, sm) + p, err := NewProvider(ctx, config.DefaultConfig.AppAuth.SelfAuthServer, sm, promutils.NewTestScope()) assert.NoError(t, err) // create a signer for rsa 256 diff --git a/flyteadmin/auth/handlers.go b/flyteadmin/auth/handlers.go index b839cf26d0..d8bc626652 100644 --- a/flyteadmin/auth/handlers.go +++ b/flyteadmin/auth/handlers.go @@ -301,23 +301,25 @@ func GetAuthenticationInterceptor(authCtx interfaces.AuthenticationContext) func fromHTTP := metautils.ExtractIncoming(ctx).Get(FromHTTPKey) isFromHTTP := fromHTTP == FromHTTPVal - identityContext, err := GRPCGetIdentityFromAccessToken(ctx, authCtx) - if err == nil { + identityContext, accessTokenErr := GRPCGetIdentityFromAccessToken(ctx, authCtx) + if accessTokenErr == nil { return SetContextForIdentity(ctx, identityContext), nil } - logger.Infof(ctx, "Failed to parse Access Token from context. Will attempt to find IDToken. Error: %v", err) + logger.Infof(ctx, "Failed to parse Access Token from context. Will attempt to find IDToken. Error: %v", accessTokenErr) - identityContext, err = GRPCGetIdentityFromIDToken(ctx, authCtx.Options().UserAuth.OpenID.ClientID, + identityContext, idTokenErr := GRPCGetIdentityFromIDToken(ctx, authCtx.Options().UserAuth.OpenID.ClientID, authCtx.OidcProvider()) - if err == nil { + if idTokenErr == nil { return SetContextForIdentity(ctx, identityContext), nil } + logger.Debugf(ctx, "Failed to parse ID Token from context. Error: %v", idTokenErr) // Only enforcement logic is present. The default case is to let things through. if (isFromHTTP && !authCtx.Options().DisableForHTTP) || (!isFromHTTP && !authCtx.Options().DisableForGrpc) { + err := fmt.Errorf("id token err: %w, access token err: %w", fmt.Errorf("access token err: %w", accessTokenErr), idTokenErr) return ctx, status.Errorf(codes.Unauthenticated, "token parse error %s", err) } diff --git a/flyteadmin/pkg/server/service.go b/flyteadmin/pkg/server/service.go index 840d0d9f17..1596d1202f 100644 --- a/flyteadmin/pkg/server/service.go +++ b/flyteadmin/pkg/server/service.go @@ -352,7 +352,7 @@ func serveGatewayInsecure(ctx context.Context, pluginRegistry *plugins.Registry, var oauth2Provider interfaces.OAuth2Provider var oauth2ResourceServer interfaces.OAuth2ResourceServer if authCfg.AppAuth.AuthServerType == authConfig.AuthorizationServerTypeSelf { - oauth2Provider, err = authzserver.NewProvider(ctx, authCfg.AppAuth.SelfAuthServer, sm) + oauth2Provider, err = authzserver.NewProvider(ctx, authCfg.AppAuth.SelfAuthServer, sm, scope.NewSubScope("auth_provider")) if err != nil { logger.Errorf(ctx, "Error creating authorization server %s", err) return err @@ -463,7 +463,7 @@ func serveGatewaySecure(ctx context.Context, pluginRegistry *plugins.Registry, c var oauth2Provider interfaces.OAuth2Provider var oauth2ResourceServer interfaces.OAuth2ResourceServer if authCfg.AppAuth.AuthServerType == authConfig.AuthorizationServerTypeSelf { - oauth2Provider, err = authzserver.NewProvider(ctx, authCfg.AppAuth.SelfAuthServer, sm) + oauth2Provider, err = authzserver.NewProvider(ctx, authCfg.AppAuth.SelfAuthServer, sm, scope.NewSubScope("auth_provider")) if err != nil { logger.Errorf(ctx, "Error creating authorization server %s", err) return err diff --git a/flyteidl/clients/go/admin/token_source_provider.go b/flyteidl/clients/go/admin/token_source_provider.go index 83df542082..4ecfa59215 100644 --- a/flyteidl/clients/go/admin/token_source_provider.go +++ b/flyteidl/clients/go/admin/token_source_provider.go @@ -9,6 +9,9 @@ import ( "strings" "sync" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/util/retry" + "golang.org/x/oauth2" "golang.org/x/oauth2/clientcredentials" @@ -167,6 +170,7 @@ func GetPKCEAuthTokenSource(ctx context.Context, pkceTokenOrchestrator pkce.Toke type ClientCredentialsTokenSourceProvider struct { ccConfig clientcredentials.Config tokenCache cache.TokenCache + cfg *Config } func NewClientCredentialsTokenSourceProvider(ctx context.Context, cfg *Config, scopes []string, tokenURL string, @@ -198,7 +202,9 @@ func NewClientCredentialsTokenSourceProvider(ctx context.Context, cfg *Config, s Scopes: scopes, EndpointParams: endpointParams, }, - tokenCache: tokenCache}, nil + tokenCache: tokenCache, + cfg: cfg, + }, nil } func (p ClientCredentialsTokenSourceProvider) GetTokenSource(ctx context.Context) (oauth2.TokenSource, error) { @@ -207,6 +213,7 @@ func (p ClientCredentialsTokenSourceProvider) GetTokenSource(ctx context.Context new: p.ccConfig.TokenSource(ctx), mu: sync.Mutex{}, tokenCache: p.tokenCache, + cfg: p.cfg, }, nil } @@ -215,6 +222,7 @@ type customTokenSource struct { mu sync.Mutex // guards everything else new oauth2.TokenSource tokenCache cache.TokenCache + cfg *Config } func (s *customTokenSource) Token() (*oauth2.Token, error) { @@ -225,10 +233,24 @@ func (s *customTokenSource) Token() (*oauth2.Token, error) { return token, nil } - token, err := s.new.Token() + totalAttempts := s.cfg.MaxRetries + 1 // Add one for initial request attempt + backoff := wait.Backoff{ + Duration: s.cfg.PerRetryTimeout.Duration, + Steps: totalAttempts, + } + var token *oauth2.Token + err := retry.OnError(backoff, func(err error) bool { + return err != nil + }, func() (err error) { + token, err = s.new.Token() + if err != nil { + logger.Infof(s.ctx, "failed to get token: %w", err) + return fmt.Errorf("failed to get token: %w", err) + } + return nil + }) if err != nil { - logger.Warnf(s.ctx, "failed to get token: %v", err) - return nil, fmt.Errorf("failed to get token: %w", err) + return nil, err } logger.Infof(s.ctx, "retrieved token with expiry %v", token.Expiry) From c68d2db16f877427685cec95ab5f4ff8aaf6b20b Mon Sep 17 00:00:00 2001 From: Haytham Abuelfutuh Date: Thu, 2 May 2024 08:27:08 -0700 Subject: [PATCH 16/25] Dask dashboard should have a separate log config Signed-off-by: Andrew Dye --- .../go/tasks/plugins/k8s/dask/config.go | 29 ++ .../go/tasks/plugins/k8s/dask/config_flags.go | 65 +++++ .../plugins/k8s/dask/config_flags_test.go | 256 ++++++++++++++++++ .../go/tasks/plugins/k8s/dask/dask.go | 2 +- 4 files changed, 351 insertions(+), 1 deletion(-) create mode 100644 flyteplugins/go/tasks/plugins/k8s/dask/config.go create mode 100755 flyteplugins/go/tasks/plugins/k8s/dask/config_flags.go create mode 100755 flyteplugins/go/tasks/plugins/k8s/dask/config_flags_test.go diff --git a/flyteplugins/go/tasks/plugins/k8s/dask/config.go b/flyteplugins/go/tasks/plugins/k8s/dask/config.go new file mode 100644 index 0000000000..aac388e116 --- /dev/null +++ b/flyteplugins/go/tasks/plugins/k8s/dask/config.go @@ -0,0 +1,29 @@ +package dask + +import ( + pluginsConfig "github.com/flyteorg/flyte/flyteplugins/go/tasks/config" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/logs" +) + +//go:generate pflags Config --default-var=defaultConfig + +var ( + defaultConfig = Config{ + Logs: logs.DefaultConfig, + } + + configSection = pluginsConfig.MustRegisterSubSection("dask", &defaultConfig) +) + +// Config is config for 'dask' plugin +type Config struct { + Logs logs.LogConfig `json:"logs,omitempty"` +} + +func GetConfig() *Config { + return configSection.GetConfig().(*Config) +} + +func SetConfig(cfg *Config) error { + return configSection.SetConfig(cfg) +} diff --git a/flyteplugins/go/tasks/plugins/k8s/dask/config_flags.go b/flyteplugins/go/tasks/plugins/k8s/dask/config_flags.go new file mode 100755 index 0000000000..03774b772b --- /dev/null +++ b/flyteplugins/go/tasks/plugins/k8s/dask/config_flags.go @@ -0,0 +1,65 @@ +// Code generated by go generate; DO NOT EDIT. +// This file was generated by robots. + +package dask + +import ( + "encoding/json" + "reflect" + + "fmt" + + "github.com/spf13/pflag" +) + +// If v is a pointer, it will get its element value or the zero value of the element type. +// If v is not a pointer, it will return it as is. +func (Config) elemValueOrNil(v interface{}) interface{} { + if t := reflect.TypeOf(v); t.Kind() == reflect.Ptr { + if reflect.ValueOf(v).IsNil() { + return reflect.Zero(t.Elem()).Interface() + } else { + return reflect.ValueOf(v).Interface() + } + } else if v == nil { + return reflect.Zero(t).Interface() + } + + return v +} + +func (Config) mustJsonMarshal(v interface{}) string { + raw, err := json.Marshal(v) + if err != nil { + panic(err) + } + + return string(raw) +} + +func (Config) mustMarshalJSON(v json.Marshaler) string { + raw, err := v.MarshalJSON() + if err != nil { + panic(err) + } + + return string(raw) +} + +// GetPFlagSet will return strongly types pflags for all fields in Config and its nested types. The format of the +// flags is json-name.json-sub-name... etc. +func (cfg Config) GetPFlagSet(prefix string) *pflag.FlagSet { + cmdFlags := pflag.NewFlagSet("Config", pflag.ExitOnError) + cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "logs.cloudwatch-enabled"), defaultConfig.Logs.IsCloudwatchEnabled, "Enable Cloudwatch Logging") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.cloudwatch-region"), defaultConfig.Logs.CloudwatchRegion, "AWS region in which Cloudwatch logs are stored.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.cloudwatch-log-group"), defaultConfig.Logs.CloudwatchLogGroup, "Log group to which streams are associated.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.cloudwatch-template-uri"), defaultConfig.Logs.CloudwatchTemplateURI, "Template Uri to use when building cloudwatch log links") + cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "logs.kubernetes-enabled"), defaultConfig.Logs.IsKubernetesEnabled, "Enable Kubernetes Logging") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.kubernetes-url"), defaultConfig.Logs.KubernetesURL, "Console URL for Kubernetes logs") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.kubernetes-template-uri"), defaultConfig.Logs.KubernetesTemplateURI, "Template Uri to use when building kubernetes log links") + cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "logs.stackdriver-enabled"), defaultConfig.Logs.IsStackDriverEnabled, "Enable Log-links to stackdriver") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.gcp-project"), defaultConfig.Logs.GCPProjectName, "Name of the project in GCP") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.stackdriver-logresourcename"), defaultConfig.Logs.StackdriverLogResourceName, "Name of the logresource in stackdriver") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "logs.stackdriver-template-uri"), defaultConfig.Logs.StackDriverTemplateURI, "Template Uri to use when building stackdriver log links") + return cmdFlags +} diff --git a/flyteplugins/go/tasks/plugins/k8s/dask/config_flags_test.go b/flyteplugins/go/tasks/plugins/k8s/dask/config_flags_test.go new file mode 100755 index 0000000000..4cd2be2b44 --- /dev/null +++ b/flyteplugins/go/tasks/plugins/k8s/dask/config_flags_test.go @@ -0,0 +1,256 @@ +// Code generated by go generate; DO NOT EDIT. +// This file was generated by robots. + +package dask + +import ( + "encoding/json" + "fmt" + "reflect" + "strings" + "testing" + + "github.com/mitchellh/mapstructure" + "github.com/stretchr/testify/assert" +) + +var dereferencableKindsConfig = map[reflect.Kind]struct{}{ + reflect.Array: {}, reflect.Chan: {}, reflect.Map: {}, reflect.Ptr: {}, reflect.Slice: {}, +} + +// Checks if t is a kind that can be dereferenced to get its underlying type. +func canGetElementConfig(t reflect.Kind) bool { + _, exists := dereferencableKindsConfig[t] + return exists +} + +// This decoder hook tests types for json unmarshaling capability. If implemented, it uses json unmarshal to build the +// object. Otherwise, it'll just pass on the original data. +func jsonUnmarshalerHookConfig(_, to reflect.Type, data interface{}) (interface{}, error) { + unmarshalerType := reflect.TypeOf((*json.Unmarshaler)(nil)).Elem() + if to.Implements(unmarshalerType) || reflect.PtrTo(to).Implements(unmarshalerType) || + (canGetElementConfig(to.Kind()) && to.Elem().Implements(unmarshalerType)) { + + raw, err := json.Marshal(data) + if err != nil { + fmt.Printf("Failed to marshal Data: %v. Error: %v. Skipping jsonUnmarshalHook", data, err) + return data, nil + } + + res := reflect.New(to).Interface() + err = json.Unmarshal(raw, &res) + if err != nil { + fmt.Printf("Failed to umarshal Data: %v. Error: %v. Skipping jsonUnmarshalHook", data, err) + return data, nil + } + + return res, nil + } + + return data, nil +} + +func decode_Config(input, result interface{}) error { + config := &mapstructure.DecoderConfig{ + TagName: "json", + WeaklyTypedInput: true, + Result: result, + DecodeHook: mapstructure.ComposeDecodeHookFunc( + mapstructure.StringToTimeDurationHookFunc(), + mapstructure.StringToSliceHookFunc(","), + jsonUnmarshalerHookConfig, + ), + } + + decoder, err := mapstructure.NewDecoder(config) + if err != nil { + return err + } + + return decoder.Decode(input) +} + +func join_Config(arr interface{}, sep string) string { + listValue := reflect.ValueOf(arr) + strs := make([]string, 0, listValue.Len()) + for i := 0; i < listValue.Len(); i++ { + strs = append(strs, fmt.Sprintf("%v", listValue.Index(i))) + } + + return strings.Join(strs, sep) +} + +func testDecodeJson_Config(t *testing.T, val, result interface{}) { + assert.NoError(t, decode_Config(val, result)) +} + +func testDecodeRaw_Config(t *testing.T, vStringSlice, result interface{}) { + assert.NoError(t, decode_Config(vStringSlice, result)) +} + +func TestConfig_GetPFlagSet(t *testing.T) { + val := Config{} + cmdFlags := val.GetPFlagSet("") + assert.True(t, cmdFlags.HasFlags()) +} + +func TestConfig_SetFlags(t *testing.T) { + actual := Config{} + cmdFlags := actual.GetPFlagSet("") + assert.True(t, cmdFlags.HasFlags()) + + t.Run("Test_logs.cloudwatch-enabled", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.cloudwatch-enabled", testValue) + if vBool, err := cmdFlags.GetBool("logs.cloudwatch-enabled"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vBool), &actual.Logs.IsCloudwatchEnabled) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.cloudwatch-region", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.cloudwatch-region", testValue) + if vString, err := cmdFlags.GetString("logs.cloudwatch-region"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.CloudwatchRegion) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.cloudwatch-log-group", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.cloudwatch-log-group", testValue) + if vString, err := cmdFlags.GetString("logs.cloudwatch-log-group"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.CloudwatchLogGroup) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.cloudwatch-template-uri", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.cloudwatch-template-uri", testValue) + if vString, err := cmdFlags.GetString("logs.cloudwatch-template-uri"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.CloudwatchTemplateURI) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.kubernetes-enabled", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.kubernetes-enabled", testValue) + if vBool, err := cmdFlags.GetBool("logs.kubernetes-enabled"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vBool), &actual.Logs.IsKubernetesEnabled) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.kubernetes-url", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.kubernetes-url", testValue) + if vString, err := cmdFlags.GetString("logs.kubernetes-url"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.KubernetesURL) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.kubernetes-template-uri", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.kubernetes-template-uri", testValue) + if vString, err := cmdFlags.GetString("logs.kubernetes-template-uri"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.KubernetesTemplateURI) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.stackdriver-enabled", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.stackdriver-enabled", testValue) + if vBool, err := cmdFlags.GetBool("logs.stackdriver-enabled"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vBool), &actual.Logs.IsStackDriverEnabled) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.gcp-project", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.gcp-project", testValue) + if vString, err := cmdFlags.GetString("logs.gcp-project"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.GCPProjectName) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.stackdriver-logresourcename", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.stackdriver-logresourcename", testValue) + if vString, err := cmdFlags.GetString("logs.stackdriver-logresourcename"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.StackdriverLogResourceName) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_logs.stackdriver-template-uri", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("logs.stackdriver-template-uri", testValue) + if vString, err := cmdFlags.GetString("logs.stackdriver-template-uri"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Logs.StackDriverTemplateURI) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) +} diff --git a/flyteplugins/go/tasks/plugins/k8s/dask/dask.go b/flyteplugins/go/tasks/plugins/k8s/dask/dask.go index d3b4ab32f1..4c9a551cba 100644 --- a/flyteplugins/go/tasks/plugins/k8s/dask/dask.go +++ b/flyteplugins/go/tasks/plugins/k8s/dask/dask.go @@ -279,7 +279,7 @@ func createJobSpec(workerSpec daskAPI.WorkerSpec, schedulerSpec daskAPI.Schedule } func (p daskResourceHandler) GetTaskPhase(ctx context.Context, pluginContext k8s.PluginContext, r client.Object) (pluginsCore.PhaseInfo, error) { - logPlugin, err := logs.InitializeLogPlugins(logs.GetLogConfig()) + logPlugin, err := logs.InitializeLogPlugins(&GetConfig().Logs) if err != nil { return pluginsCore.PhaseInfoUndefined, err } From d00a1592619e7ca6a3bbbf78c1b74669503b6af2 Mon Sep 17 00:00:00 2001 From: Jan Fiedler <89976021+fiedlerNr9@users.noreply.github.com> Date: Thu, 2 May 2024 22:33:34 +0200 Subject: [PATCH 17/25] adjust Dask LogName to (Dask Runner Logs) Signed-off-by: Andrew Dye --- flyteplugins/go/tasks/plugins/k8s/dask/dask.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flyteplugins/go/tasks/plugins/k8s/dask/dask.go b/flyteplugins/go/tasks/plugins/k8s/dask/dask.go index 4c9a551cba..ae68a4c760 100644 --- a/flyteplugins/go/tasks/plugins/k8s/dask/dask.go +++ b/flyteplugins/go/tasks/plugins/k8s/dask/dask.go @@ -296,7 +296,7 @@ func (p daskResourceHandler) GetTaskPhase(ctx context.Context, pluginContext k8s tasklog.Input{ Namespace: job.ObjectMeta.Namespace, PodName: job.Status.JobRunnerPodName, - LogName: "(User logs)", + LogName: "(Dask Runner Logs)", TaskExecutionID: taskExecID, }, ) From 6db54583c2c867d3a411dfc17a7dce4b097ec6ef Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Mon, 13 May 2024 10:54:25 -0700 Subject: [PATCH 18/25] Fix k3d local setup prefix I was trying to use `setup_local_dev.sh`, and it wasn't working out of the box. Looks like it expects `k3d-` prefix for the kubecontext Ran `setup_local_dev.sh` Signed-off-by: Andrew Dye --- script/setup_local_dev.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/script/setup_local_dev.sh b/script/setup_local_dev.sh index 9fbefeb9c1..cde45bf330 100644 --- a/script/setup_local_dev.sh +++ b/script/setup_local_dev.sh @@ -147,7 +147,7 @@ fi echo -e "\nSetting kubeconfig and kubectl context" export KUBECONFIG=$KUBECONFIG:"${K3D_KUBECONFIG_FILE_PATH}" -kubectl config set-context $K3D_CLUSTER_NAME +kubectl config set-context k3d-$K3D_CLUSTER_NAME DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )" HELM_CHART="${DIR}/../charts/flyte-deps" From 0221c54c4f3a40c3f09940c4c01c1221384f7405 Mon Sep 17 00:00:00 2001 From: Dan Rammer Date: Fri, 7 Jun 2024 09:31:17 -0500 Subject: [PATCH 19/25] Override ArrayNode log links with map plugin This PR adds a configuration option to override ArrayNode log links with those defined in the map plugin. The map plugin contains it's own configuration for log links, which may differ from those defined on the PodPlugin. ArrayNode, executing subNodes as regular tasks (ie. using the PodPlugin) means that it uses the default PodPlugin log templates. Signed-off-by: Andrew Dye --- .../plugins/array/k8s/subtask_exec_context.go | 10 +- .../array/k8s/subtask_exec_context_test.go | 8 +- .../pkg/controller/config/config.go | 1 + .../pkg/controller/config/config_flags.go | 1 + .../controller/config/config_flags_test.go | 14 ++ .../controller/nodes/array/event_recorder.go | 127 ++++++++++++++++++ .../nodes/array/event_recorder_test.go | 79 +++++++++++ .../pkg/controller/nodes/array/handler.go | 1 + .../pkg/controller/workflow/executor_test.go | 10 ++ 9 files changed, 242 insertions(+), 9 deletions(-) diff --git a/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context.go b/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context.go index 77b3ac6501..b76fe70d28 100644 --- a/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context.go +++ b/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context.go @@ -175,7 +175,7 @@ func (s SubTaskExecutionID) GetLogSuffix() string { return fmt.Sprintf(" #%d-%d-%d", s.taskRetryAttempt, s.executionIndex, s.subtaskRetryAttempt) } -var logTemplateRegexes = struct { +var LogTemplateRegexes = struct { ExecutionIndex *regexp.Regexp ParentName *regexp.Regexp RetryAttempt *regexp.Regexp @@ -189,17 +189,17 @@ var logTemplateRegexes = struct { func (s SubTaskExecutionID) TemplateVarsByScheme() []tasklog.TemplateVar { return []tasklog.TemplateVar{ - {Regex: logTemplateRegexes.ParentName, Value: s.parentName}, + {Regex: LogTemplateRegexes.ParentName, Value: s.parentName}, { - Regex: logTemplateRegexes.ExecutionIndex, + Regex: LogTemplateRegexes.ExecutionIndex, Value: strconv.FormatUint(uint64(s.executionIndex), 10), }, { - Regex: logTemplateRegexes.RetryAttempt, + Regex: LogTemplateRegexes.RetryAttempt, Value: strconv.FormatUint(s.subtaskRetryAttempt, 10), }, { - Regex: logTemplateRegexes.ParentRetryAttempt, + Regex: LogTemplateRegexes.ParentRetryAttempt, Value: strconv.FormatUint(uint64(s.taskRetryAttempt), 10), }, } diff --git a/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context_test.go b/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context_test.go index 103980fab0..a7f5aa20b4 100644 --- a/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context_test.go +++ b/flyteplugins/go/tasks/plugins/array/k8s/subtask_exec_context_test.go @@ -37,10 +37,10 @@ func TestSubTaskExecutionContext(t *testing.T) { assert.Equal(t, storage.DataReference("/raw_prefix/5/1"), stCtx.OutputWriter().GetRawOutputPrefix()) assert.Equal(t, []tasklog.TemplateVar{ - {Regex: logTemplateRegexes.ParentName, Value: "notfound"}, - {Regex: logTemplateRegexes.ExecutionIndex, Value: "0"}, - {Regex: logTemplateRegexes.RetryAttempt, Value: "1"}, - {Regex: logTemplateRegexes.ParentRetryAttempt, Value: "0"}, + {Regex: LogTemplateRegexes.ParentName, Value: "notfound"}, + {Regex: LogTemplateRegexes.ExecutionIndex, Value: "0"}, + {Regex: LogTemplateRegexes.RetryAttempt, Value: "1"}, + {Regex: LogTemplateRegexes.ParentRetryAttempt, Value: "0"}, }, stCtx.TaskExecutionMetadata().GetTaskExecutionID().(SubTaskExecutionID).TemplateVarsByScheme(), ) diff --git a/flytepropeller/pkg/controller/config/config.go b/flytepropeller/pkg/controller/config/config.go index 488ada1127..4801b8993a 100644 --- a/flytepropeller/pkg/controller/config/config.go +++ b/flytepropeller/pkg/controller/config/config.go @@ -348,6 +348,7 @@ const ( type ArrayNodeConfig struct { EventVersion int `json:"event-version" pflag:",ArrayNode eventing version. 0 => legacy (drop-in replacement for maptask), 1 => new"` DefaultParallelismBehavior ParallelismBehavior `json:"default-parallelism-behavior" pflag:",Default parallelism behavior for array nodes"` + UseMapPluginLogs bool `json:"use-map-plugin-logs" pflag:",Override subNode log links with those configured for the map plugin logs"` } // GetConfig extracts the Configuration from the global config module in flytestdlib and returns the corresponding type-casted object. diff --git a/flytepropeller/pkg/controller/config/config_flags.go b/flytepropeller/pkg/controller/config/config_flags.go index d8496a56fe..5d26351908 100755 --- a/flytepropeller/pkg/controller/config/config_flags.go +++ b/flytepropeller/pkg/controller/config/config_flags.go @@ -112,6 +112,7 @@ func (cfg Config) GetPFlagSet(prefix string) *pflag.FlagSet { cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "node-execution-worker-count"), defaultConfig.NodeExecutionWorkerCount, "Number of workers to evaluate node executions, currently only used for array nodes") cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "array-node-config.event-version"), defaultConfig.ArrayNode.EventVersion, "ArrayNode eventing version. 0 => legacy (drop-in replacement for maptask), 1 => new") cmdFlags.String(fmt.Sprintf("%v%v", prefix, "array-node-config.default-parallelism-behavior"), defaultConfig.ArrayNode.DefaultParallelismBehavior, "Default parallelism behavior for array nodes") + cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "array-node-config.use-map-plugin-logs"), defaultConfig.ArrayNode.UseMapPluginLogs, "Override subNode log links with those configured for the map plugin logs") cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "literal-offloading-config.Enabled"), defaultConfig.LiteralOffloadingConfig.Enabled, "") cmdFlags.StringToString(fmt.Sprintf("%v%v", prefix, "literal-offloading-config.supported-sdk-versions"), defaultConfig.LiteralOffloadingConfig.SupportedSDKVersions, "Maps flytekit and union SDK names to minimum supported version that can handle reading offloaded literals.") cmdFlags.Int64(fmt.Sprintf("%v%v", prefix, "literal-offloading-config.min-size-in-mb-for-offloading"), defaultConfig.LiteralOffloadingConfig.MinSizeInMBForOffloading, "Size of a literal at which to trigger offloading") diff --git a/flytepropeller/pkg/controller/config/config_flags_test.go b/flytepropeller/pkg/controller/config/config_flags_test.go index 109dc47b28..380a4b940b 100755 --- a/flytepropeller/pkg/controller/config/config_flags_test.go +++ b/flytepropeller/pkg/controller/config/config_flags_test.go @@ -967,6 +967,20 @@ func TestConfig_SetFlags(t *testing.T) { } }) }) + t.Run("Test_array-node-config.use-map-plugin-logs", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("array-node-config.use-map-plugin-logs", testValue) + if vBool, err := cmdFlags.GetBool("array-node-config.use-map-plugin-logs"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vBool), &actual.ArrayNode.UseMapPluginLogs) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) t.Run("Test_literal-offloading-config.Enabled", func(t *testing.T) { t.Run("Override", func(t *testing.T) { diff --git a/flytepropeller/pkg/controller/nodes/array/event_recorder.go b/flytepropeller/pkg/controller/nodes/array/event_recorder.go index ac1ad3e39f..999b383f39 100644 --- a/flytepropeller/pkg/controller/nodes/array/event_recorder.go +++ b/flytepropeller/pkg/controller/nodes/array/event_recorder.go @@ -10,14 +10,44 @@ import ( idlcore "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/logs" + pluginscore "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/encoding" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/tasklog" + mapplugin "github.com/flyteorg/flyte/flyteplugins/go/tasks/plugins/array/k8s" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/plugins/k8s/pod" "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/config" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/common" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/interfaces" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/task" + "github.com/flyteorg/flyte/flytestdlib/logger" ) +type taskExecutionID struct { + pluginscore.TaskExecutionID + + generatedName string + id *idlcore.TaskExecutionIdentifier + nodeID string +} + +func (t *taskExecutionID) GetGeneratedName() string { + return t.generatedName +} + +func (t *taskExecutionID) GetID() idlcore.TaskExecutionIdentifier { + return *t.id +} + +func (t *taskExecutionID) GetGeneratedNameWith(minLength, maxLength int) (string, error) { + return "", nil +} + +func (t *taskExecutionID) GetUniqueNodeID() string { + return t.nodeID +} + type arrayEventRecorder interface { interfaces.EventRecorder process(ctx context.Context, nCtx interfaces.NodeExecutionContext, index int, retryAttempt uint32) error @@ -83,7 +113,25 @@ func (e *externalResourcesEventRecorder) process(ctx context.Context, nCtx inter }) } + var mapLogPlugin tasklog.Plugin + if config.GetConfig().ArrayNode.UseMapPluginLogs { + mapLogPlugin, err = logs.InitializeLogPlugins(&mapplugin.GetConfig().LogConfig.Config) + if err != nil { + logger.Warnf(ctx, "failed to initialize log plugin with error:%v", err) + } + } + for _, taskExecutionEvent := range e.taskEvents { + if mapLogPlugin != nil && len(taskExecutionEvent.Logs) > 0 { + // override log links for subNode execution with map plugin + logs, err := getPluginLogs(mapLogPlugin, nCtx, index, retryAttempt) + if err != nil { + logger.Warnf(ctx, "failed to compute logs for ArrayNode:%s index:%d retryAttempt:%d with error:%v", nCtx.NodeID(), index, retryAttempt, err) + } else { + taskExecutionEvent.Logs = logs + } + } + for _, log := range taskExecutionEvent.Logs { log.Name = fmt.Sprintf("%s-%d", log.Name, index) } @@ -213,6 +261,85 @@ func newArrayEventRecorder(eventRecorder interfaces.EventRecorder) arrayEventRec } } +func getPluginLogs(logPlugin tasklog.Plugin, nCtx interfaces.NodeExecutionContext, index int, retryAttempt uint32) ([]*idlcore.TaskLog, error) { + subNodeSpec := nCtx.Node().GetArrayNode().GetSubNodeSpec() + + // retrieve taskTemplate from subNode + taskID := subNodeSpec.GetTaskID() + executableTask, err := nCtx.ExecutionContext().GetTask(*taskID) + if err != nil { + return nil, err + } + + taskTemplate := executableTask.CoreTask() + + // build TaskExecutionID + taskExecutionIdentifier := &idlcore.TaskExecutionIdentifier{ + TaskId: taskTemplate.GetId(), // use taskID from subNodeSpec + RetryAttempt: nCtx.CurrentAttempt(), + NodeExecutionId: nCtx.NodeExecutionMetadata().GetNodeExecutionID(), // use node metadata from ArrayNode + } + + nodeID := nCtx.NodeID() + if nCtx.ExecutionContext().GetEventVersion() != v1alpha1.EventVersion0 { + var err error + nodeID, err = common.GenerateUniqueID(nCtx.ExecutionContext().GetParentInfo(), nCtx.NodeID()) + if err != nil { + return nil, err + } + } + + length := task.IDMaxLength + if l := pod.DefaultPodPlugin.GetProperties().GeneratedNameMaxLength; l != nil { + length = *l + } + + uniqueID, err := encoding.FixedLengthUniqueIDForParts(length, []string{nCtx.NodeExecutionMetadata().GetOwnerID().Name, nodeID, strconv.Itoa(int(nCtx.CurrentAttempt()))}) + if err != nil { + return nil, err + } + + taskExecID := &taskExecutionID{ + generatedName: uniqueID, + id: taskExecutionIdentifier, + nodeID: nodeID, + } + + // compute podName and containerName + stCtx := mapplugin.NewSubTaskExecutionID(taskExecID, index, uint64(retryAttempt)) + + podName := stCtx.GetGeneratedName() + containerName := stCtx.GetGeneratedName() + + // initialize map plugin specific LogTemplateVars + extraLogTemplateVars := []tasklog.TemplateVar{ + { + Regex: mapplugin.LogTemplateRegexes.ExecutionIndex, + Value: strconv.FormatUint(uint64(index), 10), + }, + { + Regex: mapplugin.LogTemplateRegexes.RetryAttempt, + Value: strconv.FormatUint(uint64(retryAttempt), 10), + }, + } + + logs, err := logPlugin.GetTaskLogs( + tasklog.Input{ + PodName: podName, + Namespace: nCtx.NodeExecutionMetadata().GetNamespace(), + ContainerName: containerName, + TaskExecutionID: taskExecID, + ExtraTemplateVars: extraLogTemplateVars, + TaskTemplate: taskTemplate, + }, + ) + if err != nil { + return nil, err + } + + return logs.TaskLogs, nil +} + func sendEvents(ctx context.Context, nCtx interfaces.NodeExecutionContext, index int, retryAttempt uint32, nodePhase idlcore.NodeExecution_Phase, taskPhase idlcore.TaskExecution_Phase, eventRecorder interfaces.EventRecorder, eventConfig *config.EventConfig) error { diff --git a/flytepropeller/pkg/controller/nodes/array/event_recorder_test.go b/flytepropeller/pkg/controller/nodes/array/event_recorder_test.go index 9d0f6faeb5..64fbff7666 100644 --- a/flytepropeller/pkg/controller/nodes/array/event_recorder_test.go +++ b/flytepropeller/pkg/controller/nodes/array/event_recorder_test.go @@ -2,9 +2,19 @@ package array import ( "context" + "testing" + "github.com/stretchr/testify/assert" + "k8s.io/apimachinery/pkg/types" + + idlcore "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/logs" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/tasklog" + "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/config" + execmocks "github.com/flyteorg/flyte/flytepropeller/pkg/controller/executors/mocks" + "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/interfaces/mocks" ) type bufferedEventRecorder struct { @@ -25,3 +35,72 @@ func (b *bufferedEventRecorder) RecordNodeEvent(ctx context.Context, nodeExecuti func newBufferedEventRecorder() *bufferedEventRecorder { return &bufferedEventRecorder{} } + +func TestGetPluginLogs(t *testing.T) { + // initialize log plugin + logConfig := &logs.LogConfig{ + Templates: []tasklog.TemplateLogPlugin{ + tasklog.TemplateLogPlugin{ + Name: "foo", + DisplayName: "bar", + TemplateURIs: []tasklog.TemplateURI{ + "/console/projects/{{.executionProject}}/domains/{{.executionDomain}}/executions/{{.executionName}}/nodeId/{{.nodeID}}/taskId/{{.taskID}}/attempt/{{.taskRetryAttempt}}/mappedIndex/{{.subtaskExecutionIndex}}/mappedAttempt/{{.subtaskRetryAttempt}}/view/logs?duration=all", + }, + }, + }, + } + + mapLogPlugin, err := logs.InitializeLogPlugins(logConfig) + assert.Nil(t, err) + + // create NodeExecutionContext + nCtx := &mocks.NodeExecutionContext{} + nCtx.OnCurrentAttempt().Return(uint32(0)) + + executionContext := &execmocks.ExecutionContext{} + executionContext.OnGetEventVersion().Return(1) + executionContext.OnGetParentInfo().Return(nil) + executionContext.OnGetTaskMatch(taskRef).Return( + &v1alpha1.TaskSpec{ + TaskTemplate: &idlcore.TaskTemplate{ + Id: &idlcore.Identifier{ + ResourceType: idlcore.ResourceType_TASK, + Project: "task_project", + Domain: "task_domain", + Name: "task_name", + Version: "task_version", + }, + }, + }, + nil, + ) + nCtx.OnExecutionContext().Return(executionContext) + + nCtx.OnNode().Return(&arrayNodeSpec) + + nodeExecutionMetadata := &mocks.NodeExecutionMetadata{} + nodeExecutionMetadata.OnGetNamespace().Return("node_namespace") + nodeExecutionMetadata.OnGetNodeExecutionID().Return(&idlcore.NodeExecutionIdentifier{ + NodeId: "node_id", + ExecutionId: &idlcore.WorkflowExecutionIdentifier{ + Project: "node_project", + Domain: "node_domain", + Name: "node_name", + }, + }) + nodeExecutionMetadata.OnGetOwnerID().Return(types.NamespacedName{ + Namespace: "wf_namespace", + Name: "wf_name", + }) + nCtx.OnNodeExecutionMetadata().Return(nodeExecutionMetadata) + + nCtx.OnNodeID().Return("foo") + + // call `getPluginLogs` + logs, err := getPluginLogs(mapLogPlugin, nCtx, 1, 0) + assert.Nil(t, err) + + assert.Equal(t, len(logConfig.Templates), len(logs)) + assert.Equal(t, "bar", logs[0].Name) + assert.Equal(t, "/console/projects/node_project/domains/node_domain/executions/node_name/nodeId/foo/taskId/task_name/attempt/0/mappedIndex/1/mappedAttempt/0/view/logs?duration=all", logs[0].Uri) +} diff --git a/flytepropeller/pkg/controller/nodes/array/handler.go b/flytepropeller/pkg/controller/nodes/array/handler.go index 6859bf46f0..797995734c 100644 --- a/flytepropeller/pkg/controller/nodes/array/handler.go +++ b/flytepropeller/pkg/controller/nodes/array/handler.go @@ -354,6 +354,7 @@ func (a *arrayNodeHandler) Handle(ctx context.Context, nCtx interfaces.NodeExecu } } } + if err := eventRecorder.process(ctx, nCtx, index, subNodeStatus.GetAttempts()); err != nil { return handler.UnknownTransition, err } diff --git a/flytepropeller/pkg/controller/workflow/executor_test.go b/flytepropeller/pkg/controller/workflow/executor_test.go index a3d028e94b..f691a0028c 100644 --- a/flytepropeller/pkg/controller/workflow/executor_test.go +++ b/flytepropeller/pkg/controller/workflow/executor_test.go @@ -40,6 +40,7 @@ import ( nodemocks "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/interfaces/mocks" recoveryMocks "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/recovery/mocks" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/subworkflow/launchplan" + taskconfig "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/task/config" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/task/fakeplugins" wfErrors "github.com/flyteorg/flyte/flytepropeller/pkg/controller/workflow/errors" execStats "github.com/flyteorg/flyte/flytepropeller/pkg/controller/workflowstore" @@ -227,6 +228,15 @@ func createTaskExecutorErrorInCheck(t assert.TestingT) pluginCore.PluginEntry { func TestWorkflowExecutor_HandleFlyteWorkflow_Error(t *testing.T) { ctx := context.Background() scope := testScope.NewSubScope("12") + + taskConfig := taskconfig.GetConfig() + taskConfig.TaskPlugins.DefaultForTaskTypes = map[string]string{ + "python-task": "pod", + "container": "pod", + "raw-container": "pod", + "sidecar": "pod", + } + store := createInmemoryDataStore(t, scope.NewSubScope("data_store")) recorder := StdOutEventRecorder() _, err := events.ConstructEventSink(ctx, &events.Config{Type: events.EventSinkLog}, scope.NewSubScope("event_sink")) From c76334542359452f5110e2b64684350dd66f9b75 Mon Sep 17 00:00:00 2001 From: Andrew Dye Date: Fri, 8 Mar 2024 12:36:36 -0800 Subject: [PATCH 20/25] Add histogram stopwatch to stow storage This change * Adds a new `HistogramStopWatch` to promutils. This [allows for aggregating latencies](https://prometheus.io/docs/practices/histograms/#quantiles) across pods and computing quantiles at query time * Adds `HistogramStopWatch` latency metrics for stow so that we can reason about storage latencies in aggregate. Existing latency metrics remain. - [x] Added unittests Signed-off-by: Andrew Dye --- .../promutils/labeled/histogram_stopwatch.go | 92 +++++ .../labeled/histogram_stopwatch_test.go | 327 ++++++++++++++++++ .../promutils/labeled/stopwatch_test.go | 26 +- flytestdlib/promutils/scope.go | 88 +++++ flytestdlib/promutils/scope_test.go | 40 +++ flytestdlib/storage/stow_store.go | 83 +++-- 6 files changed, 612 insertions(+), 44 deletions(-) create mode 100644 flytestdlib/promutils/labeled/histogram_stopwatch.go create mode 100644 flytestdlib/promutils/labeled/histogram_stopwatch_test.go diff --git a/flytestdlib/promutils/labeled/histogram_stopwatch.go b/flytestdlib/promutils/labeled/histogram_stopwatch.go new file mode 100644 index 0000000000..9beea73e63 --- /dev/null +++ b/flytestdlib/promutils/labeled/histogram_stopwatch.go @@ -0,0 +1,92 @@ +package labeled + +import ( + "context" + "time" + + "k8s.io/apimachinery/pkg/util/sets" + + "github.com/flyteorg/flyte/flytestdlib/contextutils" + "github.com/flyteorg/flyte/flytestdlib/promutils" +) + +type HistogramStopWatch struct { + *promutils.HistogramStopWatchVec + promutils.HistogramStopWatch + + labels []contextutils.Key +} + +// Start creates a new Instance of the HistogramStopWatch called a Timer that is closeable/stoppable. +func (c HistogramStopWatch) Start(ctx context.Context) Timer { + w, err := c.HistogramStopWatchVec.GetMetricWith(contextutils.Values(ctx, c.labels...)) + if err != nil { + panic(err.Error()) + } + + if c.HistogramStopWatch.Observer == nil { + return w.Start() + } + + return timer{ + Timers: []Timer{ + w.Start(), + c.HistogramStopWatch.Start(), + }, + } +} + +// Observe observes specified duration between the start and end time. The data point will be labeled with values from context. +// See labeled.SetMetricsKeys for information about how to configure that. +func (c HistogramStopWatch) Observe(ctx context.Context, start, end time.Time) { + w, err := c.HistogramStopWatchVec.GetMetricWith(contextutils.Values(ctx, c.labels...)) + if err != nil { + panic(err.Error()) + } + w.Observe(start, end) + + if c.HistogramStopWatch.Observer != nil { + c.HistogramStopWatch.Observe(start, end) + } +} + +// Time observes the elapsed duration since the creation of the timer. The timer is created using a StopWatch. +// The data point will be labeled with values from context. See labeled.SetMetricsKeys for information about to +// configure that. +func (c HistogramStopWatch) Time(ctx context.Context, f func()) { + t := c.Start(ctx) + f() + t.Stop() +} + +// NewHistogramStopWatch creates a new labeled HistogramStopWatch. Label keys must be set before instantiating a counter. See labeled.SetMetricsKeys +// for information about how to configure that. +func NewHistogramStopWatch(name, description string, scope promutils.Scope, opts ...MetricOption) HistogramStopWatch { + if len(metricKeys) == 0 { + panic(ErrNeverSet) + } + + sw := HistogramStopWatch{} + + name = promutils.SanitizeMetricName(name) + for _, opt := range opts { + if _, emitUnableMetric := opt.(EmitUnlabeledMetricOption); emitUnableMetric { + sw.HistogramStopWatch = scope.MustNewHistogramStopWatch(GetUnlabeledMetricName(name), description) + } else if additionalLabels, casted := opt.(AdditionalLabelsOption); casted { + // compute unique labels + labelSet := sets.NewString(metricStringKeys...) + labelSet.Insert(additionalLabels.Labels...) + labels := labelSet.List() + + sw.HistogramStopWatchVec = scope.MustNewHistogramStopWatchVec(name, description, labels...) + sw.labels = contextutils.MetricKeysFromStrings(labels) + } + } + + if sw.HistogramStopWatchVec == nil { + sw.HistogramStopWatchVec = scope.MustNewHistogramStopWatchVec(name, description, metricStringKeys...) + sw.labels = metricKeys + } + + return sw +} diff --git a/flytestdlib/promutils/labeled/histogram_stopwatch_test.go b/flytestdlib/promutils/labeled/histogram_stopwatch_test.go new file mode 100644 index 0000000000..9ef344ee3f --- /dev/null +++ b/flytestdlib/promutils/labeled/histogram_stopwatch_test.go @@ -0,0 +1,327 @@ +package labeled + +import ( + "context" + "strings" + "testing" + "time" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/assert" + + "github.com/flyteorg/flyte/flytestdlib/contextutils" + "github.com/flyteorg/flyte/flytestdlib/promutils" +) + +func ExampleHistogramStopWatch_Start() { + ctx := context.Background() + stopWatch := NewHistogramStopWatch("test", "this is an example histogram stopwatch", promutils.NewTestScope()) + { + timer := stopWatch.Start(ctx) + defer timer.Stop() + + // An operation you want to measure the time for. + time.Sleep(time.Second) + } +} + +func TestLabeledHistogramStopWatch(t *testing.T) { + UnsetMetricKeys() + assert.NotPanics(t, func() { + SetMetricKeys(contextutils.ProjectKey, contextutils.DomainKey, contextutils.WorkflowIDKey, contextutils.TaskIDKey) + }) + + t.Run("Labeled", func(t *testing.T) { + scope := promutils.NewScope("testscope_hist_stopwatch") + s := NewHistogramStopWatch("s1", "some desc", scope) + assert.NotNil(t, s) + + ctx := context.TODO() + const header = ` + # HELP testscope_hist_stopwatch:s1 some desc + # TYPE testscope_hist_stopwatch:s1 histogram + ` + + w := s.Start(ctx) + w.Stop() + var expected = ` + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 + testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 + ` + err := testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) + assert.NoError(t, err) + + ctx = contextutils.WithProjectDomain(ctx, "project", "domain") + w = s.Start(ctx) + w.Stop() + expected = ` + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 + testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""} 0 + testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""} 1 + ` + err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) + assert.NoError(t, err) + + now := time.Now() + s.Observe(ctx, now, now.Add(time.Minute)) + expected = ` + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 + testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"} 2 + testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""} 60 + testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""} 2 + ` + err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) + assert.NoError(t, err) + + s.Time(ctx, func() { + // Do nothing + }) + expected = ` + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 + testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"} 2 + testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"} 3 + testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""} 60 + testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""} 3 + ` + err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) + assert.NoError(t, err) + }) + + t.Run("Unlabeled", func(t *testing.T) { + scope := promutils.NewScope("testscope_hist_stopwatch") + s := NewHistogramStopWatch("s2", "some desc", scope, EmitUnlabeledMetric) + assert.NotNil(t, s) + + ctx := context.TODO() + const header = ` + # HELP testscope_hist_stopwatch:s2_unlabeled some desc + # TYPE testscope_hist_stopwatch:s2_unlabeled histogram + ` + + w := s.Start(ctx) + w.Stop() + var expected = ` + testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.005"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.01"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.025"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.05"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.1"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.25"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.5"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="1"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="2.5"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="5"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="10"} 1 + testscope_hist_stopwatch:s2_unlabeled_bucket{le="+Inf"} 1 + testscope_hist_stopwatch:s2_unlabeled_sum 0 + testscope_hist_stopwatch:s2_unlabeled_count 1 + ` + err := testutil.CollectAndCompare(s.HistogramStopWatch.Observer.(prometheus.Histogram), strings.NewReader(header+expected)) + assert.NoError(t, err) + }) + + t.Run("AdditionalLabels", func(t *testing.T) { + scope := promutils.NewScope("testscope_hist_stopwatch") + opts := AdditionalLabelsOption{Labels: []string{contextutils.ProjectKey.String(), contextutils.ExecIDKey.String()}} + s := NewHistogramStopWatch("s3", "some desc", scope, opts) + assert.NotNil(t, s) + + ctx := context.TODO() + const header = ` + # HELP testscope_hist_stopwatch:s3 some desc + # TYPE testscope_hist_stopwatch:s3 histogram + ` + + w := s.Start(ctx) + w.Stop() + var expected = ` + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""} 0 + testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""} 1 + ` + err := testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) + assert.NoError(t, err) + + ctx = contextutils.WithProjectDomain(ctx, "project", "domain") + w = s.Start(ctx) + w.Stop() + expected = ` + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""} 0 + testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="",project="project",task="",wf=""} 0 + testscope_hist_stopwatch:s3_count{domain="domain",exec_id="",project="project",task="",wf=""} 1 + ` + err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) + assert.NoError(t, err) + + ctx = contextutils.WithExecutionID(ctx, "exec_id") + w = s.Start(ctx) + w.Stop() + expected = ` + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""} 0 + testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="",project="project",task="",wf=""} 0 + testscope_hist_stopwatch:s3_count{domain="domain",exec_id="",project="project",task="",wf=""} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.005"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.01"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.025"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.05"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.25"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="1"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="2.5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="5"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="10"} 1 + testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="+Inf"} 1 + testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="exec_id",project="project",task="",wf=""} 0 + testscope_hist_stopwatch:s3_count{domain="domain",exec_id="exec_id",project="project",task="",wf=""} 1 + ` + err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) + assert.NoError(t, err) + }) +} diff --git a/flytestdlib/promutils/labeled/stopwatch_test.go b/flytestdlib/promutils/labeled/stopwatch_test.go index 640ea94222..e759048399 100644 --- a/flytestdlib/promutils/labeled/stopwatch_test.go +++ b/flytestdlib/promutils/labeled/stopwatch_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" @@ -114,23 +115,22 @@ func TestLabeledStopWatch(t *testing.T) { assert.NotNil(t, s) ctx := context.TODO() - /*const header = ` - # HELP testscope_stopwatch:s2_m some desc - # TYPE testscope_stopwatch:s2_m summary - `*/ + const header = ` + # HELP testscope_stopwatch:s2_unlabeled_m some desc + # TYPE testscope_stopwatch:s2_unlabeled_m summary + ` w := s.Start(ctx) w.Stop() - // promutils.StopWatch does not implement prometheus.Collector - /*var expected = ` - testscope_stopwatch:s2_m{quantile="0.5"} 0 - testscope_stopwatch:s2_m{quantile="0.9"} 0 - testscope_stopwatch:s2_m{quantile="0.99"} 0 - testscope_stopwatch:s2_m_sum 0 - testscope_stopwatch:s2_m_count 1 + var expected = ` + testscope_stopwatch:s2_unlabeled_m{quantile="0.5"} 0 + testscope_stopwatch:s2_unlabeled_m{quantile="0.9"} 0 + testscope_stopwatch:s2_unlabeled_m{quantile="0.99"} 0 + testscope_stopwatch:s2_unlabeled_m_sum 0 + testscope_stopwatch:s2_unlabeled_m_count 1 ` - err := testutil.CollectAndCompare(s.StopWatch, strings.NewReader(header+expected)) - assert.NoError(t, err)*/ + err := testutil.CollectAndCompare(s.StopWatch.Observer.(prometheus.Summary), strings.NewReader(header+expected)) + assert.NoError(t, err) }) t.Run("AdditionalLabels", func(t *testing.T) { diff --git a/flytestdlib/promutils/scope.go b/flytestdlib/promutils/scope.go index 8525747564..13b74f290d 100644 --- a/flytestdlib/promutils/scope.go +++ b/flytestdlib/promutils/scope.go @@ -86,6 +86,42 @@ func (s StopWatchVec) GetMetricWith(labels prometheus.Labels) (StopWatch, error) }, nil } +// HistogramStopWatch implements a stopwatch style interface that works with prometheus histogram +// NOTE: Do not create a HistogramStopWatch object by hand, use a Scope to get a new instance of the StopWatch object +type HistogramStopWatch struct { + StopWatch +} + +// HistogramStopWatchVec implements a stopwatch style interface that works with prometheus histogram +// NOTE: Do not create a HistogramStopWatchVec object by hand, use a Scope to get a new instance of the StopWatch object +type HistogramStopWatchVec struct { + *prometheus.HistogramVec + outputScale time.Duration +} + +// Gets a concrete StopWatch instance that can be used to start a timer and record observations. +func (h HistogramStopWatchVec) WithLabelValues(values ...string) HistogramStopWatch { + return HistogramStopWatch{ + StopWatch: StopWatch{ + Observer: h.HistogramVec.WithLabelValues(values...), + outputScale: h.outputScale, + }, + } +} + +func (h HistogramStopWatchVec) GetMetricWith(labels prometheus.Labels) (HistogramStopWatch, error) { + hVec, err := h.HistogramVec.GetMetricWith(labels) + if err != nil { + return HistogramStopWatch{}, err + } + return HistogramStopWatch{ + StopWatch{ + Observer: hVec, + outputScale: h.outputScale, + }, + }, nil +} + // Timer is a stoppable instance of a StopWatch or a Timer // A Timer can only be stopped. On stopping it will output the elapsed duration to prometheus type Timer struct { @@ -194,6 +230,20 @@ type Scope interface { NewStopWatchVec(name, description string, scale time.Duration, labelNames ...string) (*StopWatchVec, error) MustNewStopWatchVec(name, description string, scale time.Duration, labelNames ...string) *StopWatchVec + // NewHistogramStopWatch is a custom wrapper to create a HistogramStopWatch object in the current Scope. + // Unlike a StopWatch, a HistogramStopWatch can be aggregated across instances. Quantiles are computed server side. + // See https://prometheus.io/docs/practices/histograms/#quantiles for tradeoffs. + // Scale is assumed to be seconds with buckets spanning 0.005s to 10s. + NewHistogramStopWatch(name, description string) (HistogramStopWatch, error) + MustNewHistogramStopWatch(name, description string) HistogramStopWatch + + // NewHistogramStopWatchVec is a custom wrapper to create a HistogramStopWatchVec object in the current Scope. + // Unlike a StopWatchVec, a HistogramStopWatchVec can be aggregated across instances. Quantiles are computed server side. + // See https://prometheus.io/docs/practices/histograms/#quantiles for tradeoffs. + // Scale is assumed to be seconds with buckets spanning 0.005s to 10s. + NewHistogramStopWatchVec(name, description string, labelNames ...string) (*HistogramStopWatchVec, error) + MustNewHistogramStopWatchVec(name, description string, labelNames ...string) *HistogramStopWatchVec + // NewSubScope creates a new subScope in case nesting is desired for metrics. This is generally useful in creating // Scoped and SubScoped metrics NewSubScope(name string) Scope @@ -419,6 +469,44 @@ func (m metricsScope) MustNewStopWatchVec(name, description string, scale time.D return s } +func (m metricsScope) NewHistogramStopWatch(name, description string) (HistogramStopWatch, error) { + h, err := m.NewHistogram(name, description) + if err != nil { + return HistogramStopWatch{}, err + } + + return HistogramStopWatch{ + StopWatch: StopWatch{ + Observer: h, + outputScale: time.Second, + }, + }, nil +} + +func (m metricsScope) MustNewHistogramStopWatch(name, description string) HistogramStopWatch { + s, err := m.NewHistogramStopWatch(name, description) + panicIfError(err) + return s +} + +func (m metricsScope) NewHistogramStopWatchVec(name, description string, labelNames ...string) (*HistogramStopWatchVec, error) { + h, err := m.NewHistogramVec(name, description, labelNames...) + if err != nil { + return &HistogramStopWatchVec{}, err + } + + return &HistogramStopWatchVec{ + HistogramVec: h, + outputScale: time.Second, + }, nil +} + +func (m metricsScope) MustNewHistogramStopWatchVec(name, description string, labelNames ...string) *HistogramStopWatchVec { + h, err := m.NewHistogramStopWatchVec(name, description, labelNames...) + panicIfError(err) + return h +} + func (m metricsScope) CurrentScope() string { return m.scope } diff --git a/flytestdlib/promutils/scope_test.go b/flytestdlib/promutils/scope_test.go index 325c160aff..734964653d 100644 --- a/flytestdlib/promutils/scope_test.go +++ b/flytestdlib/promutils/scope_test.go @@ -171,3 +171,43 @@ func TestStopWatchVec_WithLabelValues(t *testing.T) { assert.NotNil(t, i.start) i.Stop() } + +func TestHistogramStopWatch_Start(t *testing.T) { + scope := NewTestScope() + stopwatch, err := scope.NewHistogramStopWatch("yt"+rand.String(3), "timer") + assert.NoError(t, err) + assert.Equal(t, time.Second, stopwatch.outputScale) + timer := stopwatch.Start() + assert.Equal(t, time.Second, timer.outputScale) + assert.NotNil(t, timer.start) +} + +func TestHistogramStopWatch_Observe(t *testing.T) { + scope := NewTestScope() + stopwatch, err := scope.NewHistogramStopWatch("yt"+rand.String(3), "timer") + assert.NoError(t, err) + assert.Equal(t, time.Second, stopwatch.outputScale) + stopwatch.Observe(time.Now(), time.Now().Add(time.Second)) +} + +func TestHistogramStopWatch_Time(t *testing.T) { + scope := NewTestScope() + stopwatch, err := scope.NewHistogramStopWatch("yt"+rand.String(3), "timer") + assert.NoError(t, err) + assert.Equal(t, time.Second, stopwatch.outputScale) + stopwatch.Time(func() { + }) +} + +func TestHistogramStopWatchVec_WithLabelValues(t *testing.T) { + scope := NewTestScope() + vec, err := scope.NewHistogramStopWatchVec("yt"+rand.String(3), "timer", "workflow", "label") + assert.NoError(t, err) + assert.Equal(t, time.Second, vec.outputScale) + stopwatch := vec.WithLabelValues("my_wf", "something") + assert.NotNil(t, stopwatch) + i := stopwatch.Start() + assert.Equal(t, time.Second, i.outputScale) + assert.NotNil(t, i.start) + i.Stop() +} diff --git a/flytestdlib/storage/stow_store.go b/flytestdlib/storage/stow_store.go index 6b731b9c86..4b8089b502 100644 --- a/flytestdlib/storage/stow_store.go +++ b/flytestdlib/storage/stow_store.go @@ -89,20 +89,25 @@ type stowMetrics struct { BadReference labeled.Counter BadContainer labeled.Counter - HeadFailure labeled.Counter - HeadLatency labeled.StopWatch + HeadFailure labeled.Counter + HeadLatency labeled.StopWatch + HeadLatencyHist labeled.HistogramStopWatch - ListFailure labeled.Counter - ListLatency labeled.StopWatch + ListFailure labeled.Counter + ListLatency labeled.StopWatch + ListLatencyHist labeled.HistogramStopWatch - ReadFailure labeled.Counter - ReadOpenLatency labeled.StopWatch + ReadFailure labeled.Counter + ReadOpenLatency labeled.StopWatch + ReadOpenLatencyHist labeled.HistogramStopWatch - WriteFailure labeled.Counter - WriteLatency labeled.StopWatch + WriteFailure labeled.Counter + WriteLatency labeled.StopWatch + WriteLatencyHist labeled.HistogramStopWatch - DeleteFailure labeled.Counter - DeleteLatency labeled.StopWatch + DeleteFailure labeled.Counter + DeleteLatency labeled.StopWatch + DeleteLatencyHist labeled.HistogramStopWatch } // StowMetadata that will be returned @@ -220,8 +225,12 @@ func (s *StowStore) Head(ctx context.Context, reference DataReference) (Metadata return nil, err } - t := s.metrics.HeadLatency.Start(ctx) + t1 := s.metrics.HeadLatency.Start(ctx) + t2 := s.metrics.HeadLatencyHist.Start(ctx) item, err := container.Item(k) + t1.Stop() + t2.Stop() + if err == nil { if _, err = item.Metadata(); err != nil { // Err will be caught below @@ -232,7 +241,6 @@ func (s *StowStore) Head(ctx context.Context, reference DataReference) (Metadata } else if metadata, err := item.Metadata(); err != nil { // Err will be caught below } else { - t.Stop() contentMD5, ok := metadata[strings.ToLower(FlyteContentMD5)].(string) if !ok { logger.Infof(ctx, "Failed to cast contentMD5 [%v] to string", contentMD5) @@ -266,7 +274,8 @@ func (s *StowStore) List(ctx context.Context, reference DataReference, maxItems return nil, NewCursorAtEnd(), err } - t := s.metrics.ListLatency.Start(ctx) + t1 := s.metrics.ListLatency.Start(ctx) + t2 := s.metrics.ListLatencyHist.Start(ctx) var stowCursor string if cursor.cursorState == AtStartCursorState { stowCursor = stow.CursorStart @@ -276,6 +285,9 @@ func (s *StowStore) List(ctx context.Context, reference DataReference, maxItems stowCursor = cursor.customPosition } items, stowCursor, err := container.Items(k, stowCursor, maxItems) + t1.Stop() + t2.Stop() + if err == nil { results := make([]DataReference, len(items)) for index, item := range items { @@ -286,7 +298,6 @@ func (s *StowStore) List(ctx context.Context, reference DataReference, maxItems } else { cursor = NewCursorFromCustomPosition(stowCursor) } - t.Stop() return results, cursor, nil } @@ -306,13 +317,16 @@ func (s *StowStore) ReadRaw(ctx context.Context, reference DataReference) (io.Re return nil, err } - t := s.metrics.ReadOpenLatency.Start(ctx) + t1 := s.metrics.ReadOpenLatency.Start(ctx) + t2 := s.metrics.ReadOpenLatencyHist.Start(ctx) item, err := container.Item(k) + t1.Stop() + t2.Stop() + if err != nil { incFailureCounterForError(ctx, s.metrics.ReadFailure, err) return nil, err } - t.Stop() sizeBytes, err := item.Size() if err != nil { @@ -340,8 +354,12 @@ func (s *StowStore) WriteRaw(ctx context.Context, reference DataReference, size return err } - t := s.metrics.WriteLatency.Start(ctx) + t1 := s.metrics.WriteLatency.Start(ctx) + t2 := s.metrics.WriteLatencyHist.Start(ctx) _, err = container.Put(k, raw, size, opts.Metadata) + t1.Stop() + t2.Stop() + if err != nil { // If this error is due to the bucket not existing, first attempt to create it and retry the getContainer call. if IsNotFound(err) || awsBucketIsNotFound(err) { @@ -356,8 +374,6 @@ func (s *StowStore) WriteRaw(ctx context.Context, reference DataReference, size } } - t.Stop() - return nil } @@ -374,8 +390,8 @@ func (s *StowStore) Delete(ctx context.Context, reference DataReference) error { return err } - t := s.metrics.DeleteLatency.Start(ctx) - defer t.Stop() + defer s.metrics.DeleteLatency.Start(ctx).Stop() + defer s.metrics.DeleteLatencyHist.Start(ctx).Stop() if err := container.RemoveItem(k); err != nil { incFailureCounterForError(ctx, s.metrics.DeleteFailure, err) @@ -474,20 +490,25 @@ func newStowMetrics(scope promutils.Scope) *stowMetrics { BadReference: labeled.NewCounter("bad_key", "Indicates the provided storage reference/key is incorrectly formatted", scope, labeled.EmitUnlabeledMetric), BadContainer: labeled.NewCounter("bad_container", "Indicates request for a container that has not been initialized", scope, labeled.EmitUnlabeledMetric), - HeadFailure: labeled.NewCounter("head_failure", "Indicates failure in HEAD for a given reference", scope, labeled.EmitUnlabeledMetric), - HeadLatency: labeled.NewStopWatch("head", "Indicates time to fetch metadata using the Head API", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + HeadFailure: labeled.NewCounter("head_failure", "Indicates failure in HEAD for a given reference", scope, labeled.EmitUnlabeledMetric), + HeadLatency: labeled.NewStopWatch("head", "Indicates time to fetch metadata using the Head API", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + HeadLatencyHist: labeled.NewHistogramStopWatch("head", "Indicates time to fetch metadata using the Head API", scope, labeled.EmitUnlabeledMetric), - ListFailure: labeled.NewCounter("list_failure", "Indicates failure in item listing for a given reference", scope, labeled.EmitUnlabeledMetric), - ListLatency: labeled.NewStopWatch("list", "Indicates time to fetch item listing using the List API", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + ListFailure: labeled.NewCounter("list_failure", "Indicates failure in item listing for a given reference", scope, labeled.EmitUnlabeledMetric), + ListLatency: labeled.NewStopWatch("list", "Indicates time to fetch item listing using the List API", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + ListLatencyHist: labeled.NewHistogramStopWatch("list", "Indicates time to fetch item listing using the List API", scope, labeled.EmitUnlabeledMetric), - ReadFailure: labeled.NewCounter("read_failure", "Indicates failure in GET for a given reference", scope, labeled.EmitUnlabeledMetric, failureTypeOption), - ReadOpenLatency: labeled.NewStopWatch("read_open", "Indicates time to first byte when reading", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + ReadFailure: labeled.NewCounter("read_failure", "Indicates failure in GET for a given reference", scope, labeled.EmitUnlabeledMetric, failureTypeOption), + ReadOpenLatency: labeled.NewStopWatch("read_open", "Indicates time to first byte when reading", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + ReadOpenLatencyHist: labeled.NewHistogramStopWatch("read_open", "Indicates time to first byte when reading", scope, labeled.EmitUnlabeledMetric), - WriteFailure: labeled.NewCounter("write_failure", "Indicates failure in storing/PUT for a given reference", scope, labeled.EmitUnlabeledMetric, failureTypeOption), - WriteLatency: labeled.NewStopWatch("write", "Time to write an object irrespective of size", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + WriteFailure: labeled.NewCounter("write_failure", "Indicates failure in storing/PUT for a given reference", scope, labeled.EmitUnlabeledMetric, failureTypeOption), + WriteLatency: labeled.NewStopWatch("write", "Time to write an object irrespective of size", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + WriteLatencyHist: labeled.NewHistogramStopWatch("write", "Time to write an object irrespective of size", scope, labeled.EmitUnlabeledMetric), - DeleteFailure: labeled.NewCounter("delete_failure", "Indicates failure in removing/DELETE for a given reference", scope, labeled.EmitUnlabeledMetric, failureTypeOption), - DeleteLatency: labeled.NewStopWatch("delete", "Time to delete an object irrespective of size", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + DeleteFailure: labeled.NewCounter("delete_failure", "Indicates failure in removing/DELETE for a given reference", scope, labeled.EmitUnlabeledMetric, failureTypeOption), + DeleteLatency: labeled.NewStopWatch("delete", "Time to delete an object irrespective of size", time.Millisecond, scope, labeled.EmitUnlabeledMetric), + DeleteLatencyHist: labeled.NewHistogramStopWatch("delete", "Time to delete an object irrespective of size", scope, labeled.EmitUnlabeledMetric), } } From b6996427b6c5be23d82ee14df3bd46ee663f60ca Mon Sep 17 00:00:00 2001 From: Iaroslav Ciupin Date: Wed, 26 Jun 2024 21:29:54 +0300 Subject: [PATCH 21/25] Fix metrics scale division in timer * Fix metrics scale division in timer Signed-off-by: Iaroslav Ciupin Signed-off-by: Andrew Dye --- datacatalog/go.mod | 17 +- datacatalog/go.sum | 34 +- flyteadmin/go.mod | 26 +- flyteadmin/go.sum | 40 +- flytecopilot/go.mod | 17 +- flytecopilot/go.sum | 34 +- flytectl/go.mod | 15 +- flytectl/go.sum | 32 +- .../clients/go/admin/auth_interceptor_test.go | 2 +- flyteidl/go.mod | 21 +- flyteidl/go.sum | 45 +- flyteplugins/go.mod | 17 +- flyteplugins/go.sum | 34 +- flytepropeller/go.mod | 17 +- flytepropeller/go.sum | 34 +- flytestdlib/go.mod | 16 +- flytestdlib/go.sum | 32 +- .../labeled/histogram_stopwatch_test.go | 513 ++++++++++-------- .../promutils/labeled/stopwatch_test.go | 203 ++++--- flytestdlib/promutils/scope.go | 2 +- flytestdlib/promutils/scope_test.go | 12 +- go.mod | 27 +- go.sum | 48 +- 23 files changed, 641 insertions(+), 597 deletions(-) diff --git a/datacatalog/go.mod b/datacatalog/go.mod index ecc793c76b..d5266cf5c6 100644 --- a/datacatalog/go.mod +++ b/datacatalog/go.mod @@ -38,7 +38,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/coocood/freecache v1.1.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/emicklei/go-restful/v3 v3.9.0 // indirect @@ -86,7 +86,6 @@ require ( github.com/mattn/go-colorable v0.1.12 // indirect github.com/mattn/go-isatty v0.0.14 // indirect github.com/mattn/go-sqlite3 v1.14.17 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect @@ -96,10 +95,10 @@ require ( github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_golang v1.16.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_golang v1.19.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/spf13/afero v1.8.2 // indirect github.com/spf13/cast v1.4.1 // indirect @@ -119,9 +118,9 @@ require ( go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.1.0 // indirect golang.org/x/crypto v0.25.0 // indirect - golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e // indirect + golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect golang.org/x/net v0.27.0 // indirect - golang.org/x/oauth2 v0.16.0 // indirect + golang.org/x/oauth2 v0.18.0 // indirect golang.org/x/sync v0.7.0 // indirect golang.org/x/sys v0.22.0 // indirect golang.org/x/term v0.22.0 // indirect @@ -132,7 +131,7 @@ require ( google.golang.org/genproto v0.0.0-20240123012728-ef4313101c80 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240123012728-ef4313101c80 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.66.4 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect diff --git a/datacatalog/go.sum b/datacatalog/go.sum index bd159896d1..2fd9567374 100644 --- a/datacatalog/go.sum +++ b/datacatalog/go.sum @@ -73,8 +73,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -301,8 +301,6 @@ github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9 github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= github.com/mattn/go-sqlite3 v1.14.17 h1:mCRHCLDUBXgpKAqIKsaAaAsrAlbkeomtRFKXh2L6YIM= github.com/mattn/go-sqlite3 v1.14.17/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -330,15 +328,15 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= @@ -439,8 +437,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e h1:+WEEuIdZHnUeJJmEUjyYC2gfUMj69yZXw17EnHg/otA= -golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e/go.mod h1:Kr81I6Kryrl9sr8s2FK3vxD90NdsKWRuOIl2O4CvYbA= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 h1:aAcj0Da7eBAtrTp03QXWvm88pSyOt+UgdZw2BFZ+lEw= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8/go.mod h1:CQ1k9gNrJ50XIzaKCRR2hssIjF07kZFEiieALBM/ARQ= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -510,8 +508,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -751,8 +749,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/flyteadmin/go.mod b/flyteadmin/go.mod index 11bce657a8..aeedace860 100644 --- a/flyteadmin/go.mod +++ b/flyteadmin/go.mod @@ -40,8 +40,8 @@ require ( github.com/ory/fosite v0.42.2 github.com/ory/x v0.0.214 github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.16.0 - github.com/prometheus/client_model v0.4.0 + github.com/prometheus/client_golang v1.19.1 + github.com/prometheus/client_model v0.6.1 github.com/robfig/cron/v3 v3.0.0 github.com/samber/lo v1.47.0 github.com/sendgrid/sendgrid-go v3.10.0+incompatible @@ -52,13 +52,13 @@ require ( go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 go.opentelemetry.io/otel v1.24.0 golang.org/x/net v0.27.0 - golang.org/x/oauth2 v0.16.0 + golang.org/x/oauth2 v0.18.0 golang.org/x/sync v0.7.0 golang.org/x/time v0.5.0 google.golang.org/api v0.155.0 google.golang.org/genproto v0.0.0-20240123012728-ef4313101c80 google.golang.org/grpc v1.62.1 - google.golang.org/protobuf v1.33.0 + google.golang.org/protobuf v1.34.1 gorm.io/driver/mysql v1.4.4 gorm.io/driver/postgres v1.5.3 gorm.io/driver/sqlite v1.5.4 @@ -89,7 +89,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/coocood/freecache v1.1.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect @@ -138,7 +138,7 @@ require ( github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/kelseyhightower/envconfig v1.4.0 // indirect - github.com/klauspost/compress v1.9.8 // indirect + github.com/klauspost/compress v1.17.8 // indirect github.com/kylelemons/godebug v1.1.0 // indirect github.com/lestrrat-go/backoff/v2 v2.0.8 // indirect github.com/lestrrat-go/blackmagic v1.0.2 // indirect @@ -147,10 +147,9 @@ require ( github.com/lestrrat-go/option v1.0.1 // indirect github.com/mailru/easyjson v0.7.7 // indirect github.com/mattn/go-colorable v0.1.12 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect + github.com/mattn/go-isatty v0.0.16 // indirect github.com/mattn/go-sqlite3 v2.0.3+incompatible // indirect github.com/mattn/goveralls v0.0.6 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mohae/deepcopy v0.0.0-20170929034955-c48cc78d4826 // indirect @@ -165,9 +164,9 @@ require ( github.com/pierrec/lz4 v2.5.2+incompatible // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect - github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 // indirect - github.com/samber/lo v1.47.0 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect + github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/sendgrid/rest v2.6.9+incompatible // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/spf13/afero v1.8.2 // indirect @@ -191,6 +190,8 @@ require ( go.opentelemetry.io/otel/sdk v1.24.0 // indirect go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.1.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.27.0 // indirect golang.org/x/crypto v0.25.0 // indirect golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect golang.org/x/sys v0.22.0 // indirect @@ -219,9 +220,6 @@ require ( github.com/bradfitz/gomemcache v0.0.0-20180710155616-bc664df96737 // indirect github.com/cloudevents/sdk-go/protocol/kafka_sarama/v2 v2.8.0 github.com/imdario/mergo v0.3.13 // indirect - github.com/prometheus/common v0.44.0 // indirect - go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.27.0 // indirect k8s.io/klog/v2 v2.100.1 // indirect k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9 // indirect k8s.io/utils v0.0.0-20230406110748-d93618cff8a2 // indirect diff --git a/flyteadmin/go.sum b/flyteadmin/go.sum index 4667cd3083..a31e680160 100644 --- a/flyteadmin/go.sum +++ b/flyteadmin/go.sum @@ -135,8 +135,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -875,8 +875,9 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.8 h1:VMAMUUOh+gaxKTMk+zqbjsSjsIcUcL/LF4o63i82QyA= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= +github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/knadh/koanf v0.14.1-0.20201201075439-e0853799f9ec/go.mod h1:H5mEFsTeWizwFXHKtsITL5ipsLTuAMQoGuQpp+1JL9U= github.com/konsorten/go-windows-terminal-sequences v0.0.0-20180402223658-b729f2633dfe/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -968,8 +969,9 @@ github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hd github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ= +github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-sqlite3 v1.10.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-sqlite3 v1.11.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= @@ -979,8 +981,6 @@ github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpe github.com/mattn/goveralls v0.0.6 h1:cr8Y0VMo/MnEZBjxNN/vh6G90SZ7IMb6lms1dzMoO+Y= github.com/mattn/goveralls v0.0.6/go.mod h1:h8b4ow6FxSPMQHF6o2ve3qsclnffZjYTNEKmLesRwqw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= @@ -1110,32 +1110,33 @@ github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXP github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v0.9.4/go.mod h1:oCXIBxdI62A4cR6aTRJCgetEjecSIYzOEaeAn4iYEpM= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190425082905-87a4384529e0/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= -github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rhnvrm/simples3 v0.5.0/go.mod h1:Y+3vYm2V7Y4VijFoJHHTrja6OgPrJ2cBti8dPGkC3sA= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= @@ -1546,8 +1547,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1647,6 +1648,7 @@ golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1921,8 +1923,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/DataDog/dd-trace-go.v1 v1.22.0/go.mod h1:DVp8HmDh8PuTu2Z0fVVlBsyWaC++fzwVCaGWylTe3tg= gopkg.in/DataDog/dd-trace-go.v1 v1.27.0/go.mod h1:Sp1lku8WJMvNV0kjDI4Ni/T7J/U3BO5ct5kEaoVU8+I= gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= diff --git a/flytecopilot/go.mod b/flytecopilot/go.mod index e1dbdc7683..f02c726c7c 100644 --- a/flytecopilot/go.mod +++ b/flytecopilot/go.mod @@ -34,7 +34,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/coocood/freecache v1.1.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/emicklei/go-restful/v3 v3.9.0 // indirect @@ -69,7 +69,6 @@ require ( github.com/mailru/easyjson v0.7.7 // indirect github.com/mattn/go-colorable v0.1.12 // indirect github.com/mattn/go-isatty v0.0.14 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect @@ -79,10 +78,10 @@ require ( github.com/pelletier/go-toml/v2 v2.0.0-beta.8 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_golang v1.16.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_golang v1.19.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/spf13/afero v1.8.2 // indirect github.com/spf13/cast v1.4.1 // indirect @@ -103,9 +102,9 @@ require ( go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.1.0 // indirect golang.org/x/crypto v0.25.0 // indirect - golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e // indirect + golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect golang.org/x/net v0.27.0 // indirect - golang.org/x/oauth2 v0.16.0 // indirect + golang.org/x/oauth2 v0.18.0 // indirect golang.org/x/sync v0.7.0 // indirect golang.org/x/sys v0.22.0 // indirect golang.org/x/term v0.22.0 // indirect @@ -117,7 +116,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240123012728-ef4313101c80 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect google.golang.org/grpc v1.62.1 // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.66.4 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect diff --git a/flytecopilot/go.sum b/flytecopilot/go.sum index 9fb93ec715..32e1806045 100644 --- a/flytecopilot/go.sum +++ b/flytecopilot/go.sum @@ -71,8 +71,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -267,8 +267,6 @@ github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mitchellh/go-ps v1.0.0 h1:i6ampVEEF4wQFF+bkYfwYgY+F/uYJDktmvLPf7qIgjc= github.com/mitchellh/go-ps v1.0.0/go.mod h1:J4lOc8z8yJs6vUwklHw2XEIiT4z4C40KtWVN3nvg8Pg= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= @@ -298,15 +296,15 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= @@ -407,8 +405,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e h1:+WEEuIdZHnUeJJmEUjyYC2gfUMj69yZXw17EnHg/otA= -golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e/go.mod h1:Kr81I6Kryrl9sr8s2FK3vxD90NdsKWRuOIl2O4CvYbA= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 h1:aAcj0Da7eBAtrTp03QXWvm88pSyOt+UgdZw2BFZ+lEw= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8/go.mod h1:CQ1k9gNrJ50XIzaKCRR2hssIjF07kZFEiieALBM/ARQ= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -478,8 +476,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -719,8 +717,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/flytectl/go.mod b/flytectl/go.mod index d783ac0513..fc40d6854b 100644 --- a/flytectl/go.mod +++ b/flytectl/go.mod @@ -35,10 +35,10 @@ require ( github.com/stretchr/testify v1.9.0 github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0 github.com/zalando/go-keyring v0.1.1 - golang.org/x/oauth2 v0.16.0 + golang.org/x/oauth2 v0.18.0 golang.org/x/text v0.16.0 google.golang.org/grpc v1.62.1 - google.golang.org/protobuf v1.33.0 + google.golang.org/protobuf v1.34.1 gopkg.in/yaml.v3 v3.0.1 gotest.tools v2.2.0+incompatible k8s.io/api v0.28.4 @@ -66,7 +66,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/containerd/console v1.0.4-0.20230313162750-1ae8d489ac81 // indirect github.com/containerd/log v0.1.0 // indirect github.com/coocood/freecache v1.1.1 // indirect @@ -121,7 +121,6 @@ require ( github.com/mattn/go-isatty v0.0.18 // indirect github.com/mattn/go-localereader v0.0.1 // indirect github.com/mattn/go-runewidth v0.0.15 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/moby/docker-image-spec v1.3.1 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect @@ -136,10 +135,10 @@ require ( github.com/pelletier/go-toml v1.9.5 // indirect github.com/pelletier/go-toml/v2 v2.0.0-beta.8 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_golang v1.16.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_golang v1.19.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect github.com/spf13/afero v1.9.2 // indirect diff --git a/flytectl/go.sum b/flytectl/go.sum index 1e3b5d7ef8..bf6cfb76aa 100644 --- a/flytectl/go.sum +++ b/flytectl/go.sum @@ -85,8 +85,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/charmbracelet/bubbles v0.18.0 h1:PYv1A036luoBGroX6VWjQIE9Syf2Wby2oOl/39KLfy0= github.com/charmbracelet/bubbles v0.18.0/go.mod h1:08qhZhtIwzgrtBjAcJnij1t1H0ZRjwHyGsy6AL11PSw= github.com/charmbracelet/bubbletea v0.25.0 h1:bAfwk7jRz7FKFl9RzlIULPkStffg5k6pNt5dywy4TcM= @@ -351,8 +351,6 @@ github.com/mattn/go-localereader v0.0.1/go.mod h1:8fBrzywKY7BI3czFoHkuzRoWE9C+Ei github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/moby/docker-image-spec v1.3.1 h1:jMKff3w6PgbfSa69GfNg+zN/XLhfXJGnEx3Nl2EsFP0= @@ -402,15 +400,15 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= @@ -420,6 +418,8 @@ github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/shamaton/msgpack/v2 v2.2.2 h1:GOIg0c9LV04VwzOOqZSrmsv/JzjNOOMxnS/HvOHGdgs= +github.com/shamaton/msgpack/v2 v2.2.2/go.mod h1:6khjYnkx73f7VQU7wjcFS9DFjs+59naVWJv1TB7qdOI= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= @@ -598,8 +598,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -846,8 +846,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/flyteidl/clients/go/admin/auth_interceptor_test.go b/flyteidl/clients/go/admin/auth_interceptor_test.go index 0f47e97b9c..b03171c825 100644 --- a/flyteidl/clients/go/admin/auth_interceptor_test.go +++ b/flyteidl/clients/go/admin/auth_interceptor_test.go @@ -343,7 +343,7 @@ func TestNewAuthInterceptorAndMaterialize(t *testing.T) { AuthType: AuthTypeClientSecret, TokenURL: fmt.Sprintf("http://localhost:%d/oauth2/token", httpPort), Scopes: []string{"all"}, - Audience: "http://localhost:30081", + Audience: fmt.Sprintf("http://localhost:%d", httpPort), AuthorizationHeader: "authorization", } diff --git a/flyteidl/go.mod b/flyteidl/go.mod index 55ec124554..01e8bf916e 100644 --- a/flyteidl/go.mod +++ b/flyteidl/go.mod @@ -5,9 +5,11 @@ go 1.22 require ( github.com/flyteorg/flyte/flytestdlib v0.0.0-00010101000000-000000000000 github.com/go-test/deep v1.0.7 + github.com/golang/glog v1.2.0 github.com/golang/protobuf v1.5.3 github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 + github.com/grpc-ecosystem/grpc-gateway v1.16.0 github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 github.com/jinzhu/copier v0.3.5 github.com/mitchellh/mapstructure v1.5.0 @@ -16,11 +18,13 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.9.0 golang.org/x/net v0.27.0 - golang.org/x/oauth2 v0.16.0 + golang.org/x/oauth2 v0.18.0 + google.golang.org/api v0.155.0 google.golang.org/genproto/googleapis/api v0.0.0-20240123012728-ef4313101c80 google.golang.org/grpc v1.62.1 - google.golang.org/protobuf v1.33.0 + google.golang.org/protobuf v1.34.1 k8s.io/apimachinery v0.28.2 + k8s.io/client-go v0.28.1 ) require ( @@ -38,7 +42,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/coocood/freecache v1.1.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/emicklei/go-restful/v3 v3.9.0 // indirect @@ -70,16 +74,15 @@ require ( github.com/mailru/easyjson v0.7.7 // indirect github.com/mattn/go-colorable v0.1.12 // indirect github.com/mattn/go-isatty v0.0.14 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/ncw/swift v1.0.53 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_golang v1.16.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_golang v1.19.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/spf13/cobra v1.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect @@ -103,7 +106,6 @@ require ( golang.org/x/term v0.22.0 // indirect golang.org/x/text v0.16.0 // indirect golang.org/x/time v0.5.0 // indirect - google.golang.org/api v0.155.0 // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto v0.0.0-20240123012728-ef4313101c80 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect @@ -111,7 +113,6 @@ require ( gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/api v0.28.2 // indirect - k8s.io/client-go v0.28.1 // indirect k8s.io/klog/v2 v2.100.1 // indirect k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9 // indirect k8s.io/utils v0.0.0-20230406110748-d93618cff8a2 // indirect diff --git a/flyteidl/go.sum b/flyteidl/go.sum index 5d5cb7e9a2..c0ce2a7fee 100644 --- a/flyteidl/go.sum +++ b/flyteidl/go.sum @@ -1,4 +1,5 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.0 h1:tpFCD7hpHFlQ8yPwT3x+QeXqc2T6+n6T+hmABHfDUSM= cloud.google.com/go v0.112.0/go.mod h1:3jEEVwZ/MHU4djK5t5RHuKOA/GbLddgTdVubX1qnPD4= cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= @@ -24,6 +25,7 @@ github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83 github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/aws/aws-sdk-go v1.44.2 h1:5VBk5r06bgxgRKVaUtm1/4NT/rtrnH2E4cnAYv5zgQc= github.com/aws/aws-sdk-go v1.44.2/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX4oIKwKHZo= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -33,8 +35,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= @@ -92,12 +94,15 @@ github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69 github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/glog v1.2.0 h1:uCdmnmatrKCgMBlM4rMuJZWOkPDqdbZPnrMXDY4gI68= +github.com/golang/glog v1.2.0/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= @@ -140,6 +145,8 @@ github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 h1:THDBEeQ9xZ8JEaCLyLQqXMMdR github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= +github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 h1:Wqo399gCIufwto+VfwCSvsnfGpF/w5E9CNxSwbpD6No= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0/go.mod h1:qmOFXW2epJhM0qSnUUYpldc7gVz2KMQwJ/QYCDIa7XU= github.com/imdario/mergo v0.3.6 h1:xTNEAn+kxVO7dTZGu0CegyqKZmoWFI0rF8UxjlB2d28= @@ -180,8 +187,6 @@ github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -205,15 +210,16 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= @@ -296,11 +302,13 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= 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-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= @@ -309,8 +317,9 @@ golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -326,6 +335,7 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -378,6 +388,7 @@ google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAs google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20240123012728-ef4313101c80 h1:KAeGQVN3M9nD0/bQXnr/ClcEMJ968gUXJQ9pwfSynuQ= google.golang.org/genproto v0.0.0-20240123012728-ef4313101c80/go.mod h1:cc8bqMqtv9gMOr0zHg2Vzff5ULhhL2IXP4sbcn32Dro= @@ -389,6 +400,7 @@ google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZi google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= google.golang.org/grpc v1.62.1 h1:B4n+nfKzOICUXMgyrNd19h/I9oH0L1pizfk1d4zSgTk= google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= @@ -403,14 +415,15 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= diff --git a/flyteplugins/go.mod b/flyteplugins/go.mod index c4287581bc..be994e9aaf 100644 --- a/flyteplugins/go.mod +++ b/flyteplugins/go.mod @@ -21,17 +21,17 @@ require ( github.com/magiconair/properties v1.8.6 github.com/mitchellh/mapstructure v1.5.0 github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.16.0 + github.com/prometheus/client_golang v1.19.1 github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.9.0 golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 golang.org/x/net v0.27.0 - golang.org/x/oauth2 v0.16.0 + golang.org/x/oauth2 v0.18.0 golang.org/x/time v0.5.0 google.golang.org/api v0.155.0 google.golang.org/grpc v1.62.1 - google.golang.org/protobuf v1.33.0 + google.golang.org/protobuf v1.34.1 gopkg.in/yaml.v2 v2.4.0 gotest.tools v2.2.0+incompatible k8s.io/api v0.28.4 @@ -61,7 +61,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/emicklei/go-restful/v3 v3.11.0 // indirect github.com/evanphx/json-patch v5.6.0+incompatible // indirect @@ -95,8 +95,7 @@ require ( github.com/kylelemons/godebug v1.1.0 // indirect github.com/mailru/easyjson v0.7.7 // indirect github.com/mattn/go-colorable v0.1.12 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/mattn/go-isatty v0.0.16 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect @@ -105,9 +104,9 @@ require ( github.com/pelletier/go-toml/v2 v2.0.0-beta.8 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/spf13/afero v1.8.2 // indirect github.com/spf13/cast v1.4.1 // indirect diff --git a/flyteplugins/go.sum b/flyteplugins/go.sum index fa26e3cfda..6a22c49d10 100644 --- a/flyteplugins/go.sum +++ b/flyteplugins/go.sum @@ -93,8 +93,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -296,10 +296,9 @@ github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ= +github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -327,15 +326,15 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1 h1:skD8MXnQMO3QGUeTKt09VOXvuch/gJh8+6q3OLm0kAQ= github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1/go.mod h1:ZqyKKvMP5nKDldQoKmur+Wcx7wVlV9Q98phFqHzr+KY= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= @@ -509,8 +508,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -565,6 +564,7 @@ golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= @@ -750,8 +750,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/flytepropeller/go.mod b/flytepropeller/go.mod index 6b55e8909e..9a5c758465 100644 --- a/flytepropeller/go.mod +++ b/flytepropeller/go.mod @@ -21,7 +21,7 @@ require ( github.com/magiconair/properties v1.8.6 github.com/mitchellh/mapstructure v1.5.0 github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.16.0 + github.com/prometheus/client_golang v1.19.1 github.com/shamaton/msgpack/v2 v2.2.2 github.com/sirupsen/logrus v1.9.3 github.com/spf13/cobra v1.7.0 @@ -34,7 +34,7 @@ require ( golang.org/x/sync v0.7.0 golang.org/x/time v0.5.0 google.golang.org/grpc v1.62.1 - google.golang.org/protobuf v1.33.0 + google.golang.org/protobuf v1.34.1 k8s.io/api v0.28.4 k8s.io/apiextensions-apiserver v0.28.4 k8s.io/apimachinery v0.28.4 @@ -68,7 +68,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/coocood/freecache v1.1.1 // indirect github.com/dask/dask-kubernetes/v2023 v2023.0.0-20230626103304-abd02cd17b26 // indirect github.com/davecgh/go-spew v1.1.1 // indirect @@ -104,8 +104,7 @@ require ( github.com/kylelemons/godebug v1.1.0 // indirect github.com/mailru/easyjson v0.7.7 // indirect github.com/mattn/go-colorable v0.1.12 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/mattn/go-isatty v0.0.16 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect @@ -115,9 +114,9 @@ require ( github.com/pelletier/go-toml/v2 v2.0.0-beta.8 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1 // indirect github.com/spf13/afero v1.8.2 // indirect github.com/spf13/cast v1.4.1 // indirect @@ -137,7 +136,7 @@ require ( go.opentelemetry.io/proto/otlp v1.1.0 // indirect golang.org/x/crypto v0.25.0 // indirect golang.org/x/net v0.27.0 // indirect - golang.org/x/oauth2 v0.16.0 // indirect + golang.org/x/oauth2 v0.18.0 // indirect golang.org/x/sys v0.22.0 // indirect golang.org/x/term v0.22.0 // indirect golang.org/x/text v0.16.0 // indirect diff --git a/flytepropeller/go.sum b/flytepropeller/go.sum index dc0ccb0464..855f446847 100644 --- a/flytepropeller/go.sum +++ b/flytepropeller/go.sum @@ -97,8 +97,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -318,10 +318,9 @@ github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ= +github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -359,15 +358,15 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1 h1:skD8MXnQMO3QGUeTKt09VOXvuch/gJh8+6q3OLm0kAQ= github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1/go.mod h1:ZqyKKvMP5nKDldQoKmur+Wcx7wVlV9Q98phFqHzr+KY= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= @@ -550,8 +549,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -612,6 +611,7 @@ golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= @@ -799,8 +799,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/flytestdlib/go.mod b/flytestdlib/go.mod index db2a030ac7..d2aefdc39f 100644 --- a/flytestdlib/go.mod +++ b/flytestdlib/go.mod @@ -21,7 +21,8 @@ require ( github.com/magiconair/properties v1.8.6 github.com/mitchellh/mapstructure v1.5.0 github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.16.0 + github.com/prometheus/client_golang v1.19.1-0.20240620110541-bccd68204bf4 + github.com/prometheus/common v0.53.0 github.com/sirupsen/logrus v1.9.3 github.com/spf13/cobra v1.7.0 github.com/spf13/pflag v1.0.5 @@ -37,7 +38,7 @@ require ( golang.org/x/time v0.5.0 golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d google.golang.org/grpc v1.62.1 - google.golang.org/protobuf v1.33.0 + google.golang.org/protobuf v1.34.1 gorm.io/driver/postgres v1.5.3 gorm.io/driver/sqlite v1.5.4 gorm.io/gorm v1.25.4 @@ -62,7 +63,7 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/emicklei/go-restful/v3 v3.9.0 // indirect github.com/evanphx/json-patch v5.6.0+incompatible // indirect @@ -97,12 +98,12 @@ require ( github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.8 // indirect github.com/kylelemons/godebug v1.1.0 // indirect github.com/mailru/easyjson v0.7.7 // indirect github.com/mattn/go-colorable v0.1.12 // indirect github.com/mattn/go-isatty v0.0.14 // indirect github.com/mattn/go-sqlite3 v1.14.17 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect @@ -111,9 +112,8 @@ require ( github.com/pelletier/go-toml/v2 v2.0.0-beta.8 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/spf13/afero v1.8.2 // indirect github.com/spf13/cast v1.4.1 // indirect github.com/spf13/jwalterweatherman v1.1.0 // indirect @@ -129,7 +129,7 @@ require ( golang.org/x/exp v0.0.0-20220722155223-a9213eeb770e // indirect golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.27.0 // indirect - golang.org/x/oauth2 v0.16.0 // indirect + golang.org/x/oauth2 v0.18.0 // indirect golang.org/x/sync v0.7.0 // indirect golang.org/x/sys v0.22.0 // indirect golang.org/x/term v0.22.0 // indirect diff --git a/flytestdlib/go.sum b/flytestdlib/go.sum index da8ec6cdc6..61fc59a3fe 100644 --- a/flytestdlib/go.sum +++ b/flytestdlib/go.sum @@ -73,8 +73,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -280,6 +280,8 @@ github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1 github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= +github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= @@ -303,8 +305,6 @@ github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9 github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= github.com/mattn/go-sqlite3 v1.14.17 h1:mCRHCLDUBXgpKAqIKsaAaAsrAlbkeomtRFKXh2L6YIM= github.com/mattn/go-sqlite3 v1.14.17/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -332,15 +332,15 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1-0.20240620110541-bccd68204bf4 h1:GCiMVi+gRj5QaXuw8Gkz71k8US0ilrLJmoG/mp5+8dI= +github.com/prometheus/client_golang v1.19.1-0.20240620110541-bccd68204bf4/go.mod h1:JJCmTHsrwjUPYl5HyuWSzf8ZNGQzncCeuj37Rby0GzI= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= @@ -514,8 +514,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -755,8 +755,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/flytestdlib/promutils/labeled/histogram_stopwatch_test.go b/flytestdlib/promutils/labeled/histogram_stopwatch_test.go index 9ef344ee3f..d4608f92c8 100644 --- a/flytestdlib/promutils/labeled/histogram_stopwatch_test.go +++ b/flytestdlib/promutils/labeled/histogram_stopwatch_test.go @@ -2,13 +2,16 @@ package labeled import ( "context" + "strconv" "strings" "testing" "time" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/common/expfmt" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/flyteorg/flyte/flytestdlib/contextutils" "github.com/flyteorg/flyte/flytestdlib/promutils" @@ -36,140 +39,139 @@ func TestLabeledHistogramStopWatch(t *testing.T) { scope := promutils.NewScope("testscope_hist_stopwatch") s := NewHistogramStopWatch("s1", "some desc", scope) assert.NotNil(t, s) + metricName := scope.CurrentScope() + "s1" ctx := context.TODO() const header = ` # HELP testscope_hist_stopwatch:s1 some desc - # TYPE testscope_hist_stopwatch:s1 histogram - ` + # TYPE testscope_hist_stopwatch:s1 histogram` w := s.Start(ctx) w.Stop() - var expected = ` - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 - testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 - ` - err := testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics := map[string]any{ + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""}`: 1, + } + assertMetrics(t, s.HistogramStopWatchVec, metricName, header, expectedMetrics) ctx = contextutils.WithProjectDomain(ctx, "project", "domain") w = s.Start(ctx) w.Stop() - expected = ` - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 - testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""} 0 - testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""} 1 - ` - err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + + expectedMetrics = map[string]any{ + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""}`: 1, + } + assertMetrics(t, s.HistogramStopWatchVec, metricName, header, expectedMetrics) now := time.Now() s.Observe(ctx, now, now.Add(time.Minute)) - expected = ` - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 - testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"} 2 - testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""} 60 - testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""} 2 - ` - err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + + expectedMetrics = map[string]any{ + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"}`: 2, + `testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""}`: 60.0, + `testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""}`: 2, + } + assertMetrics(t, s.HistogramStopWatchVec, metricName, header, expectedMetrics) s.Time(ctx, func() { // Do nothing }) - expected = ` - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""} 0 - testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""} 1 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"} 2 - testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"} 3 - testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""} 60 - testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""} 3 - ` - err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + + expectedMetrics = map[string]any{ + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="",project="",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s1_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s1_count{domain="",project="",task="",wf=""}`: 1, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.005"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.01"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.025"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.05"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.1"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.25"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="0.5"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="1"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="2.5"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="5"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="10"}`: 2, + `testscope_hist_stopwatch:s1_bucket{domain="domain",project="project",task="",wf="",le="+Inf"}`: 3, + `testscope_hist_stopwatch:s1_sum{domain="domain",project="project",task="",wf=""}`: 60.0, + `testscope_hist_stopwatch:s1_count{domain="domain",project="project",task="",wf=""}`: 3, + } + assertMetrics(t, s.HistogramStopWatchVec, metricName, header, expectedMetrics) }) t.Run("Unlabeled", func(t *testing.T) { @@ -180,29 +182,27 @@ func TestLabeledHistogramStopWatch(t *testing.T) { ctx := context.TODO() const header = ` # HELP testscope_hist_stopwatch:s2_unlabeled some desc - # TYPE testscope_hist_stopwatch:s2_unlabeled histogram - ` + # TYPE testscope_hist_stopwatch:s2_unlabeled histogram` w := s.Start(ctx) w.Stop() - var expected = ` - testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.005"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.01"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.025"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.05"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.1"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.25"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.5"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="1"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="2.5"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="5"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="10"} 1 - testscope_hist_stopwatch:s2_unlabeled_bucket{le="+Inf"} 1 - testscope_hist_stopwatch:s2_unlabeled_sum 0 - testscope_hist_stopwatch:s2_unlabeled_count 1 - ` - err := testutil.CollectAndCompare(s.HistogramStopWatch.Observer.(prometheus.Histogram), strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics := map[string]any{ + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.005"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.01"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.025"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.05"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.1"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.25"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="0.5"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="1"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="2.5"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="5"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="10"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_bucket{le="+Inf"}`: 1, + `testscope_hist_stopwatch:s2_unlabeled_sum`: 0.0, + `testscope_hist_stopwatch:s2_unlabeled_count`: 1, + } + assertMetrics(t, s.HistogramStopWatch.Observer.(prometheus.Histogram), "testscope_hist_stopwatch:s2_unlabeled", header, expectedMetrics) }) t.Run("AdditionalLabels", func(t *testing.T) { @@ -210,118 +210,163 @@ func TestLabeledHistogramStopWatch(t *testing.T) { opts := AdditionalLabelsOption{Labels: []string{contextutils.ProjectKey.String(), contextutils.ExecIDKey.String()}} s := NewHistogramStopWatch("s3", "some desc", scope, opts) assert.NotNil(t, s) + metricName := scope.CurrentScope() + "s3" ctx := context.TODO() const header = ` # HELP testscope_hist_stopwatch:s3 some desc - # TYPE testscope_hist_stopwatch:s3 histogram - ` + # TYPE testscope_hist_stopwatch:s3 histogram` w := s.Start(ctx) w.Stop() - var expected = ` - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""} 0 - testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""} 1 - ` - err := testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics := map[string]any{ + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""}`: 1, + } + assertMetrics(t, s.HistogramStopWatchVec, metricName, header, expectedMetrics) ctx = contextutils.WithProjectDomain(ctx, "project", "domain") w = s.Start(ctx) w.Stop() - expected = ` - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""} 0 - testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="",project="project",task="",wf=""} 0 - testscope_hist_stopwatch:s3_count{domain="domain",exec_id="",project="project",task="",wf=""} 1 - ` - err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics = map[string]any{ + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="",project="project",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s3_count{domain="domain",exec_id="",project="project",task="",wf=""}`: 1, + } + assertMetrics(t, s.HistogramStopWatchVec, metricName, header, expectedMetrics) ctx = contextutils.WithExecutionID(ctx, "exec_id") w = s.Start(ctx) w.Stop() - expected = ` - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""} 0 - testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="",project="project",task="",wf=""} 0 - testscope_hist_stopwatch:s3_count{domain="domain",exec_id="",project="project",task="",wf=""} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.005"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.01"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.025"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.05"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.25"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="1"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="2.5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="5"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="10"} 1 - testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="+Inf"} 1 - testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="exec_id",project="project",task="",wf=""} 0 - testscope_hist_stopwatch:s3_count{domain="domain",exec_id="exec_id",project="project",task="",wf=""} 1 - ` - err = testutil.CollectAndCompare(s.HistogramStopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics = map[string]any{ + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="",exec_id="",project="",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s3_sum{domain="",exec_id="",project="",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s3_count{domain="",exec_id="",project="",task="",wf=""}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="",project="project",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="",project="project",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s3_count{domain="domain",exec_id="",project="project",task="",wf=""}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.005"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.01"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.025"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.05"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.25"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="0.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="1"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="2.5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="5"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="10"}`: 1, + `testscope_hist_stopwatch:s3_bucket{domain="domain",exec_id="exec_id",project="project",task="",wf="",le="+Inf"}`: 1, + `testscope_hist_stopwatch:s3_sum{domain="domain",exec_id="exec_id",project="project",task="",wf=""}`: 0.0, + `testscope_hist_stopwatch:s3_count{domain="domain",exec_id="exec_id",project="project",task="",wf=""}`: 1, + } + assertMetrics(t, s.HistogramStopWatchVec, metricName, header, expectedMetrics) }) } + +func assertMetrics(t *testing.T, c prometheus.Collector, metricName, expectedHeader string, expectedMetrics map[string]any) { + t.Helper() + metricBytes, err := testutil.CollectAndFormat(c, expfmt.TypeTextPlain, metricName) + require.NoError(t, err) + require.NotEmptyf(t, metricBytes, "empty `%q` metric", metricName) + + actual := strings.Split(strings.TrimSpace(string(metricBytes)), "\n") + n := len(actual) + + expected := strings.Split(strings.TrimSpace(expectedHeader), "\n") + require.Len(t, expected, 2, "wrong number of expected header lines") + + for i := 0; i < n; i++ { + line := actual[i] + + if strings.HasPrefix(line, "#") { + if i != 0 && i != 1 { + require.Failf(t, "wrong format", "comment line %q on wrong place", line) + } + assert.Equal(t, strings.TrimSpace(expected[i]), actual[i]) + continue + } + + lineSplt := strings.Split(line, " ") + if len(lineSplt) != 2 { + require.Failf(t, "metric line has wrong format", "metric %s has line %q with wrong format", metricName, line) + } + + key := lineSplt[0] + expectedValue, ok := expectedMetrics[key] + require.Truef(t, ok, "missing expected %q metric", key) + + switch expectedValue.(type) { + case int, int8, int16, int32, int64: + actualValue, err := strconv.Atoi(lineSplt[1]) + require.NoError(t, err) + assert.Equal(t, expectedValue, actualValue) + case float32, float64: + actualValue, err := strconv.ParseFloat(lineSplt[1], 64) + require.NoError(t, err) + assert.InDeltaf(t, expectedValue, actualValue, 0.001, "metric %q has wrong value", key) + assert.Greaterf(t, actualValue, expectedValue, "actual value of %q should be slightly greater than expected", key) + default: + require.Fail(t, "unsupported expected value type") + } + } +} diff --git a/flytestdlib/promutils/labeled/stopwatch_test.go b/flytestdlib/promutils/labeled/stopwatch_test.go index e759048399..4d94caabf7 100644 --- a/flytestdlib/promutils/labeled/stopwatch_test.go +++ b/flytestdlib/promutils/labeled/stopwatch_test.go @@ -2,12 +2,10 @@ package labeled import ( "context" - "strings" "testing" "time" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "github.com/flyteorg/flyte/flytestdlib/contextutils" @@ -36,77 +34,73 @@ func TestLabeledStopWatch(t *testing.T) { scope := promutils.NewScope("testscope_stopwatch") s := NewStopWatch("s1", "some desc", time.Minute, scope) assert.NotNil(t, s) + metricName := scope.CurrentScope() + "s1_m" ctx := context.TODO() const header = ` # HELP testscope_stopwatch:s1_m some desc - # TYPE testscope_stopwatch:s1_m summary - ` + # TYPE testscope_stopwatch:s1_m summary` w := s.Start(ctx) w.Stop() - var expected = ` - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""} 0 - testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""} 1 - ` - err := testutil.CollectAndCompare(s.StopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics := map[string]any{ + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""}`: 1, + } + assertMetrics(t, s.StopWatchVec, metricName, header, expectedMetrics) ctx = contextutils.WithProjectDomain(ctx, "project", "domain") w = s.Start(ctx) w.Stop() - expected = ` - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""} 0 - testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""} 1 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s1_m_sum{domain="domain",project="project",task="",wf=""} 0 - testscope_stopwatch:s1_m_count{domain="domain",project="project",task="",wf=""} 1 - ` - err = testutil.CollectAndCompare(s.StopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics = map[string]any{ + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""}`: 1, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s1_m_sum{domain="domain",project="project",task="",wf=""}`: 0.0, + `testscope_stopwatch:s1_m_count{domain="domain",project="project",task="",wf=""}`: 1, + } + assertMetrics(t, s.StopWatchVec, metricName, header, expectedMetrics) now := time.Now() s.Observe(ctx, now, now.Add(time.Minute)) - expected = ` - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""} 0 - testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""} 1 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.9"} 1 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.99"} 1 - testscope_stopwatch:s1_m_sum{domain="domain",project="project",task="",wf=""} 1 - testscope_stopwatch:s1_m_count{domain="domain",project="project",task="",wf=""} 2 - ` - err = testutil.CollectAndCompare(s.StopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics = map[string]any{ + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""}`: 1, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.9"}`: 1, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.99"}`: 1, + `testscope_stopwatch:s1_m_sum{domain="domain",project="project",task="",wf=""}`: 1.0, + `testscope_stopwatch:s1_m_count{domain="domain",project="project",task="",wf=""}`: 2, + } + assertMetrics(t, s.StopWatchVec, metricName, header, expectedMetrics) s.Time(ctx, func() { // Do nothing }) - expected = ` - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""} 0 - testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""} 1 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.9"} 1 - testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.99"} 1 - testscope_stopwatch:s1_m_sum{domain="domain",project="project",task="",wf=""} 1 - testscope_stopwatch:s1_m_count{domain="domain",project="project",task="",wf=""} 3 - ` - err = testutil.CollectAndCompare(s.StopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics = map[string]any{ + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s1_m{domain="",project="",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s1_m_sum{domain="",project="",task="",wf=""}`: 0.0, + `testscope_stopwatch:s1_m_count{domain="",project="",task="",wf=""}`: 1, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.9"}`: 1, + `testscope_stopwatch:s1_m{domain="domain",project="project",task="",wf="",quantile="0.99"}`: 1, + `testscope_stopwatch:s1_m_sum{domain="domain",project="project",task="",wf=""}`: 1.0, + `testscope_stopwatch:s1_m_count{domain="domain",project="project",task="",wf=""}`: 3, + } + assertMetrics(t, s.StopWatchVec, metricName, header, expectedMetrics) }) t.Run("Unlabeled", func(t *testing.T) { @@ -122,15 +116,14 @@ func TestLabeledStopWatch(t *testing.T) { w := s.Start(ctx) w.Stop() - var expected = ` - testscope_stopwatch:s2_unlabeled_m{quantile="0.5"} 0 - testscope_stopwatch:s2_unlabeled_m{quantile="0.9"} 0 - testscope_stopwatch:s2_unlabeled_m{quantile="0.99"} 0 - testscope_stopwatch:s2_unlabeled_m_sum 0 - testscope_stopwatch:s2_unlabeled_m_count 1 - ` - err := testutil.CollectAndCompare(s.StopWatch.Observer.(prometheus.Summary), strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics := map[string]any{ + `testscope_stopwatch:s2_unlabeled_m{quantile="0.5"}`: 0.0, + `testscope_stopwatch:s2_unlabeled_m{quantile="0.9"}`: 0.0, + `testscope_stopwatch:s2_unlabeled_m{quantile="0.99"}`: 0.0, + `testscope_stopwatch:s2_unlabeled_m_sum`: 0.0, + `testscope_stopwatch:s2_unlabeled_m_count`: 1, + } + assertMetrics(t, s.StopWatch.Observer.(prometheus.Summary), "testscope_stopwatch:s2_unlabeled_m", header, expectedMetrics) }) t.Run("AdditionalLabels", func(t *testing.T) { @@ -138,6 +131,7 @@ func TestLabeledStopWatch(t *testing.T) { opts := AdditionalLabelsOption{Labels: []string{contextutils.ProjectKey.String(), contextutils.ExecIDKey.String()}} s := NewStopWatch("s3", "some desc", time.Minute, scope, opts) assert.NotNil(t, s) + metricName := scope.CurrentScope() + "s3_m" ctx := context.TODO() const header = ` @@ -147,55 +141,52 @@ func TestLabeledStopWatch(t *testing.T) { w := s.Start(ctx) w.Stop() - var expected = ` - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s3_m_sum{domain="",exec_id="",project="",task="",wf=""} 0 - testscope_stopwatch:s3_m_count{domain="",exec_id="",project="",task="",wf=""} 1 - ` - err := testutil.CollectAndCompare(s.StopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics := map[string]any{ + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s3_m_sum{domain="",exec_id="",project="",task="",wf=""}`: 0.0, + `testscope_stopwatch:s3_m_count{domain="",exec_id="",project="",task="",wf=""}`: 1, + } + assertMetrics(t, s.StopWatchVec, metricName, header, expectedMetrics) ctx = contextutils.WithProjectDomain(ctx, "project", "domain") w = s.Start(ctx) w.Stop() - expected = ` - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s3_m_sum{domain="",exec_id="",project="",task="",wf=""} 0 - testscope_stopwatch:s3_m_count{domain="",exec_id="",project="",task="",wf=""} 1 - testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s3_m_sum{domain="domain",exec_id="",project="project",task="",wf=""} 0 - testscope_stopwatch:s3_m_count{domain="domain",exec_id="",project="project",task="",wf=""} 1 - ` - err = testutil.CollectAndCompare(s.StopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics = map[string]any{ + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s3_m_sum{domain="",exec_id="",project="",task="",wf=""}`: 0.0, + `testscope_stopwatch:s3_m_count{domain="",exec_id="",project="",task="",wf=""}`: 1, + `testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s3_m_sum{domain="domain",exec_id="",project="project",task="",wf=""}`: 0.0, + `testscope_stopwatch:s3_m_count{domain="domain",exec_id="",project="project",task="",wf=""}`: 1, + } + assertMetrics(t, s.StopWatchVec, metricName, header, expectedMetrics) ctx = contextutils.WithExecutionID(ctx, "exec_id") w = s.Start(ctx) w.Stop() - expected = ` - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s3_m_sum{domain="",exec_id="",project="",task="",wf=""} 0 - testscope_stopwatch:s3_m_count{domain="",exec_id="",project="",task="",wf=""} 1 - testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s3_m_sum{domain="domain",exec_id="",project="project",task="",wf=""} 0 - testscope_stopwatch:s3_m_count{domain="domain",exec_id="",project="project",task="",wf=""} 1 - testscope_stopwatch:s3_m{domain="domain",exec_id="exec_id",project="project",task="",wf="",quantile="0.5"} 0 - testscope_stopwatch:s3_m{domain="domain",exec_id="exec_id",project="project",task="",wf="",quantile="0.9"} 0 - testscope_stopwatch:s3_m{domain="domain",exec_id="exec_id",project="project",task="",wf="",quantile="0.99"} 0 - testscope_stopwatch:s3_m_sum{domain="domain",exec_id="exec_id",project="project",task="",wf=""} 0 - testscope_stopwatch:s3_m_count{domain="domain",exec_id="exec_id",project="project",task="",wf=""} 1 - ` - err = testutil.CollectAndCompare(s.StopWatchVec, strings.NewReader(header+expected)) - assert.NoError(t, err) + expectedMetrics = map[string]any{ + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s3_m{domain="",exec_id="",project="",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s3_m_sum{domain="",exec_id="",project="",task="",wf=""}`: 0.0, + `testscope_stopwatch:s3_m_count{domain="",exec_id="",project="",task="",wf=""}`: 1, + `testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s3_m{domain="domain",exec_id="",project="project",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s3_m_sum{domain="domain",exec_id="",project="project",task="",wf=""}`: 0.0, + `testscope_stopwatch:s3_m_count{domain="domain",exec_id="",project="project",task="",wf=""}`: 1, + `testscope_stopwatch:s3_m{domain="domain",exec_id="exec_id",project="project",task="",wf="",quantile="0.5"}`: 0.0, + `testscope_stopwatch:s3_m{domain="domain",exec_id="exec_id",project="project",task="",wf="",quantile="0.9"}`: 0.0, + `testscope_stopwatch:s3_m{domain="domain",exec_id="exec_id",project="project",task="",wf="",quantile="0.99"}`: 0.0, + `testscope_stopwatch:s3_m_sum{domain="domain",exec_id="exec_id",project="project",task="",wf=""}`: 0.0, + `testscope_stopwatch:s3_m_count{domain="domain",exec_id="exec_id",project="project",task="",wf=""}`: 1, + } + assertMetrics(t, s.StopWatchVec, metricName, header, expectedMetrics) }) } diff --git a/flytestdlib/promutils/scope.go b/flytestdlib/promutils/scope.go index 13b74f290d..1e3d027901 100644 --- a/flytestdlib/promutils/scope.go +++ b/flytestdlib/promutils/scope.go @@ -138,7 +138,7 @@ func (s Timer) Stop() float64 { s.timer.Observe(0) return 0 } - scaled := float64(observed / outputScaleDuration) + scaled := float64(observed) / float64(outputScaleDuration) s.timer.Observe(scaled) return scaled } diff --git a/flytestdlib/promutils/scope_test.go b/flytestdlib/promutils/scope_test.go index 734964653d..6f5aff61bf 100644 --- a/flytestdlib/promutils/scope_test.go +++ b/flytestdlib/promutils/scope_test.go @@ -60,7 +60,7 @@ func TestMetricsScope(t *testing.T) { } t.Run("Counter", func(t *testing.T) { m := s.MustNewCounter("xc", description) - assert.Equal(t, `Desc{fqName: "test:xc", help: "some x", constLabels: {}, variableLabels: []}`, m.Desc().String()) + assert.Equal(t, `Desc{fqName: "test:xc", help: "some x", constLabels: {}, variableLabels: {}}`, m.Desc().String()) mv := s.MustNewCounterVec("xcv", description) assert.NotNil(t, mv) assert.Panics(t, func() { @@ -73,7 +73,7 @@ func TestMetricsScope(t *testing.T) { t.Run("Histogram", func(t *testing.T) { m := s.MustNewHistogram("xh", description) - assert.Equal(t, `Desc{fqName: "test:xh", help: "some x", constLabels: {}, variableLabels: []}`, m.Desc().String()) + assert.Equal(t, `Desc{fqName: "test:xh", help: "some x", constLabels: {}, variableLabels: {}}`, m.Desc().String()) mv := s.MustNewHistogramVec("xhv", description) assert.NotNil(t, mv) assert.Panics(t, func() { @@ -92,10 +92,10 @@ func TestMetricsScope(t *testing.T) { t.Run("Summary", func(t *testing.T) { m := s.MustNewSummary("xs", description) - assert.Equal(t, `Desc{fqName: "test:xs", help: "some x", constLabels: {}, variableLabels: []}`, m.Desc().String()) + assert.Equal(t, `Desc{fqName: "test:xs", help: "some x", constLabels: {}, variableLabels: {}}`, m.Desc().String()) mco, err := s.NewSummaryWithOptions("xsco", description, SummaryOptions{Objectives: map[float64]float64{0.5: 0.05, 1.0: 0.0}}) assert.Nil(t, err) - assert.Equal(t, `Desc{fqName: "test:xsco", help: "some x", constLabels: {}, variableLabels: []}`, mco.Desc().String()) + assert.Equal(t, `Desc{fqName: "test:xsco", help: "some x", constLabels: {}, variableLabels: {}}`, mco.Desc().String()) mv := s.MustNewSummaryVec("xsv", description) assert.NotNil(t, mv) assert.Panics(t, func() { @@ -108,7 +108,7 @@ func TestMetricsScope(t *testing.T) { t.Run("Gauge", func(t *testing.T) { m := s.MustNewGauge("xg", description) - assert.Equal(t, `Desc{fqName: "test:xg", help: "some x", constLabels: {}, variableLabels: []}`, m.Desc().String()) + assert.Equal(t, `Desc{fqName: "test:xg", help: "some x", constLabels: {}, variableLabels: {}}`, m.Desc().String()) mv := s.MustNewGaugeVec("xgv", description) assert.NotNil(t, mv) assert.Panics(t, func() { @@ -123,7 +123,7 @@ func TestMetricsScope(t *testing.T) { m := s.MustNewStopWatch("xt", description, time.Second) asDesc, ok := m.Observer.(prometheus.Metric) assert.True(t, ok) - assert.Equal(t, `Desc{fqName: "test:xt_s", help: "some x", constLabels: {}, variableLabels: []}`, asDesc.Desc().String()) + assert.Equal(t, `Desc{fqName: "test:xt_s", help: "some x", constLabels: {}, variableLabels: {}}`, asDesc.Desc().String()) assert.Panics(t, func() { _ = s.MustNewStopWatch("xt", description, time.Second) }) diff --git a/go.mod b/go.mod index 16fa54c50c..68535b90b4 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/flyteorg/flyte/flytepropeller v0.0.0-00010101000000-000000000000 github.com/flyteorg/flyte/flytestdlib v0.0.0-00010101000000-000000000000 github.com/golang/glog v1.2.0 - github.com/prometheus/client_golang v1.16.0 + github.com/prometheus/client_golang v1.19.1 github.com/spf13/cobra v1.7.0 github.com/spf13/pflag v1.0.5 golang.org/x/sync v0.7.0 @@ -48,7 +48,7 @@ require ( github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/cloudevents/sdk-go/binding/format/protobuf/v2 v2.14.0 // indirect github.com/cloudevents/sdk-go/protocol/kafka_sarama/v2 v2.8.0 // indirect github.com/cloudevents/sdk-go/v2 v2.15.2 // indirect @@ -58,8 +58,8 @@ require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect github.com/dgraph-io/ristretto v0.0.3 // indirect - github.com/eapache/go-resiliency v1.2.0 // indirect - github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect + github.com/eapache/go-resiliency v1.3.0 // indirect + github.com/eapache/go-xerial-snappy v0.0.0-20230111030713-bf00bc1b83b6 // indirect github.com/eapache/queue v1.1.0 // indirect github.com/emicklei/go-restful/v3 v3.12.0 // indirect github.com/evanphx/json-patch v5.6.0+incompatible // indirect @@ -102,7 +102,7 @@ require ( github.com/grpc-ecosystem/go-grpc-prometheus v1.2.1-0.20210315223345-82c243799c99 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 // indirect github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69 // indirect - github.com/hashicorp/go-uuid v1.0.2 // indirect + github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/hashicorp/golang-lru v0.5.4 // indirect github.com/hashicorp/hcl v1.0.0 // indirect github.com/imdario/mergo v0.3.13 // indirect @@ -115,14 +115,14 @@ require ( github.com/jackc/pgservicefile v0.0.0-20231201235250-de7065d80cb9 // indirect github.com/jackc/pgx/v5 v5.5.5 // indirect github.com/jackc/puddle/v2 v2.2.1 // indirect - github.com/jcmturner/gofork v1.0.0 // indirect + github.com/jcmturner/gofork v1.7.6 // indirect github.com/jinzhu/inflection v1.0.0 // indirect github.com/jinzhu/now v1.1.5 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/kelseyhightower/envconfig v1.4.0 // indirect - github.com/klauspost/compress v1.9.8 // indirect + github.com/klauspost/compress v1.17.8 // indirect github.com/kubeflow/common v0.4.3 // indirect github.com/kubeflow/training-operator v1.5.0-rc.0 // indirect github.com/kylelemons/godebug v1.1.0 // indirect @@ -138,7 +138,6 @@ require ( github.com/mattn/go-isatty v0.0.16 // indirect github.com/mattn/go-sqlite3 v2.0.3+incompatible // indirect github.com/mattn/goveralls v0.0.6 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect @@ -158,11 +157,11 @@ require ( github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.4.0 // indirect - github.com/prometheus/common v0.44.0 // indirect - github.com/prometheus/procfs v0.10.1 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1 // indirect - github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 // indirect + github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/robfig/cron/v3 v3.0.0 // indirect github.com/samber/lo v1.47.0 // indirect github.com/sendgrid/rest v2.6.9+incompatible // indirect @@ -198,7 +197,7 @@ require ( golang.org/x/crypto v0.25.0 // indirect golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect golang.org/x/net v0.27.0 // indirect - golang.org/x/oauth2 v0.16.0 // indirect + golang.org/x/oauth2 v0.18.0 // indirect golang.org/x/sys v0.22.0 // indirect golang.org/x/term v0.22.0 // indirect golang.org/x/text v0.16.0 // indirect @@ -211,7 +210,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240123012728-ef4313101c80 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect google.golang.org/grpc v1.62.1 // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.66.4 // indirect gopkg.in/jcmturner/aescts.v1 v1.0.1 // indirect diff --git a/go.sum b/go.sum index 813ac168a1..8845c21df0 100644 --- a/go.sum +++ b/go.sum @@ -156,8 +156,8 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -229,10 +229,12 @@ github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:Htrtb github.com/dustin/go-humanize v0.0.0-20180713052910-9f541cc9db5d/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= -github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q= github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= -github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= +github.com/eapache/go-resiliency v1.3.0 h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0= +github.com/eapache/go-resiliency v1.3.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/go-xerial-snappy v0.0.0-20230111030713-bf00bc1b83b6 h1:8yY/I9ndfrgrXUbOGObLHKBR4Fl3nZXwM2c7OYTT8hM= +github.com/eapache/go-xerial-snappy v0.0.0-20230111030713-bf00bc1b83b6/go.mod h1:YvSRo5mw33fLEx1+DlK6L2VV43tJt5Eyel9n9XBcR+0= github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/elastic/go-sysinfo v1.1.1/go.mod h1:i1ZYdU10oLNfRzq4vq62BEwD2fH8KaWh6eh0ikPT9F0= @@ -779,8 +781,9 @@ github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtng github.com/hashicorp/go-hclog v0.9.2/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ= github.com/hashicorp/go-retryablehttp v0.6.8/go.mod h1:vAew36LZh98gCBJNLH42IQ1ER/9wtLZZ8meHqQvEYWY= github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= +github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= 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 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= @@ -849,8 +852,9 @@ github.com/jackc/puddle/v2 v2.2.1 h1:RhxXJtFG022u4ibrCSMSiu5aOq1i77R3OHKNJj77OAk github.com/jackc/puddle/v2 v2.2.1/go.mod h1:vriiEXHvEE654aYKXXjOvZM39qJ0q+azkZFrfEOc3H4= github.com/jandelgado/gcov2lcov v1.0.4-0.20210120124023-b83752c6dc08/go.mod h1:NnSxK6TMlg1oGDBfGelGbjgorT5/L3cchlbtgFYZSss= github.com/jcmturner/gofork v0.0.0-20190328161633-dc7c13fece03/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= -github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8= github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= +github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg= +github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jinzhu/copier v0.3.5 h1:GlvfUwHk62RokgqVNvYsku0TATCF7bAHVwEXoBh3iJg= github.com/jinzhu/copier v0.3.5/go.mod h1:DfbEm0FYsaqBcKcFuvmOZb218JkPGtvSHsKg8S8hyyg= @@ -901,8 +905,9 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.8 h1:VMAMUUOh+gaxKTMk+zqbjsSjsIcUcL/LF4o63i82QyA= github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= +github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/knadh/koanf v0.14.1-0.20201201075439-e0853799f9ec/go.mod h1:H5mEFsTeWizwFXHKtsITL5ipsLTuAMQoGuQpp+1JL9U= github.com/konsorten/go-windows-terminal-sequences v0.0.0-20180402223658-b729f2633dfe/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -1010,8 +1015,6 @@ github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpe github.com/mattn/goveralls v0.0.6 h1:cr8Y0VMo/MnEZBjxNN/vh6G90SZ7IMb6lms1dzMoO+Y= github.com/mattn/goveralls v0.0.6/go.mod h1:h8b4ow6FxSPMQHF6o2ve3qsclnffZjYTNEKmLesRwqw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= -github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= @@ -1144,34 +1147,35 @@ github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXP github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v0.9.4/go.mod h1:oCXIBxdI62A4cR6aTRJCgetEjecSIYzOEaeAn4iYEpM= -github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= -github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= +github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= -github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= -github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190425082905-87a4384529e0/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1 h1:skD8MXnQMO3QGUeTKt09VOXvuch/gJh8+6q3OLm0kAQ= github.com/ray-project/kuberay/ray-operator v1.1.0-rc.1/go.mod h1:ZqyKKvMP5nKDldQoKmur+Wcx7wVlV9Q98phFqHzr+KY= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= -github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 h1:MkV+77GLUNo5oJ0jf870itWm3D0Sjh7+Za9gazKc5LQ= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rhnvrm/simples3 v0.5.0/go.mod h1:Y+3vYm2V7Y4VijFoJHHTrja6OgPrJ2cBti8dPGkC3sA= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= @@ -1582,8 +1586,8 @@ golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= -golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1958,8 +1962,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/DataDog/dd-trace-go.v1 v1.22.0/go.mod h1:DVp8HmDh8PuTu2Z0fVVlBsyWaC++fzwVCaGWylTe3tg= gopkg.in/DataDog/dd-trace-go.v1 v1.27.0/go.mod h1:Sp1lku8WJMvNV0kjDI4Ni/T7J/U3BO5ct5kEaoVU8+I= gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= From 04c4a0410d334d7ac488e800678e3556f6360e87 Mon Sep 17 00:00:00 2001 From: Iaroslav Ciupin Date: Thu, 25 Jul 2024 09:19:02 +0300 Subject: [PATCH 22/25] CreateDownloadLink: Head before signing Signed-off-by: Andrew Dye --- flyteadmin/dataproxy/service.go | 12 +++++- flyteadmin/dataproxy/service_test.go | 59 ++++++++++++++++++++++++++++ 2 files changed, 70 insertions(+), 1 deletion(-) diff --git a/flyteadmin/dataproxy/service.go b/flyteadmin/dataproxy/service.go index cb20075a2e..c02fa3699f 100644 --- a/flyteadmin/dataproxy/service.go +++ b/flyteadmin/dataproxy/service.go @@ -182,7 +182,17 @@ func (s Service) CreateDownloadLink(ctx context.Context, req *service.CreateDown return nil, errors.NewFlyteAdminErrorf(codes.Internal, "no deckUrl found for request [%+v]", req) } - signedURLResp, err := s.dataStore.CreateSignedURL(ctx, storage.DataReference(nativeURL), storage.SignedURLProperties{ + ref := storage.DataReference(nativeURL) + meta, err := s.dataStore.Head(ctx, ref) + if err != nil { + return nil, errors.NewFlyteAdminErrorf(codes.Internal, "failed to head object before signing url. Error: %v", err) + } + + if !meta.Exists() { + return nil, errors.NewFlyteAdminErrorf(codes.NotFound, "object not found") + } + + signedURLResp, err := s.dataStore.CreateSignedURL(ctx, ref, storage.SignedURLProperties{ Scope: stow.ClientMethodGet, ExpiresIn: req.ExpiresIn.AsDuration(), }) diff --git a/flyteadmin/dataproxy/service_test.go b/flyteadmin/dataproxy/service_test.go index fb7a956a27..4c3f3ea720 100644 --- a/flyteadmin/dataproxy/service_test.go +++ b/flyteadmin/dataproxy/service_test.go @@ -4,12 +4,15 @@ import ( "bytes" "context" "crypto/md5" // #nosec + "fmt" "net/url" "testing" "time" "github.com/golang/protobuf/proto" "github.com/stretchr/testify/assert" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "google.golang.org/protobuf/types/known/durationpb" commonMocks "github.com/flyteorg/flyte/flyteadmin/pkg/common/mocks" @@ -157,6 +160,15 @@ func TestCreateUploadLocationMore(t *testing.T) { }) } +type testMetadata struct { + storage.Metadata + exists bool +} + +func (t testMetadata) Exists() bool { + return t.exists +} + func TestCreateDownloadLink(t *testing.T) { dataStore := commonMocks.GetMockStorageClient() nodeExecutionManager := &mocks.MockNodeExecutionManager{} @@ -179,7 +191,30 @@ func TestCreateDownloadLink(t *testing.T) { assert.Error(t, err) }) + t.Run("item not found", func(t *testing.T) { + dataStore.ComposedProtobufStore.(*commonMocks.TestDataStore).HeadCb = func(ctx context.Context, ref storage.DataReference) (storage.Metadata, error) { + return testMetadata{exists: false}, nil + } + + _, err = s.CreateDownloadLink(context.Background(), &service.CreateDownloadLinkRequest{ + ArtifactType: service.ArtifactType_ARTIFACT_TYPE_DECK, + Source: &service.CreateDownloadLinkRequest_NodeExecutionId{ + NodeExecutionId: &core.NodeExecutionIdentifier{}, + }, + ExpiresIn: durationpb.New(time.Hour), + }) + + st, ok := status.FromError(err) + assert.True(t, ok) + assert.Equal(t, codes.NotFound, st.Code()) + assert.Equal(t, "object not found", st.Message()) + }) + t.Run("valid config", func(t *testing.T) { + dataStore.ComposedProtobufStore.(*commonMocks.TestDataStore).HeadCb = func(ctx context.Context, ref storage.DataReference) (storage.Metadata, error) { + return testMetadata{exists: true}, nil + } + _, err = s.CreateDownloadLink(context.Background(), &service.CreateDownloadLinkRequest{ ArtifactType: service.ArtifactType_ARTIFACT_TYPE_DECK, Source: &service.CreateDownloadLinkRequest_NodeExecutionId{ @@ -187,10 +222,34 @@ func TestCreateDownloadLink(t *testing.T) { }, ExpiresIn: durationpb.New(time.Hour), }) + assert.NoError(t, err) }) + t.Run("head failed", func(t *testing.T) { + dataStore.ComposedProtobufStore.(*commonMocks.TestDataStore).HeadCb = func(ctx context.Context, ref storage.DataReference) (storage.Metadata, error) { + return testMetadata{}, fmt.Errorf("head fail") + } + + _, err = s.CreateDownloadLink(context.Background(), &service.CreateDownloadLinkRequest{ + ArtifactType: service.ArtifactType_ARTIFACT_TYPE_DECK, + Source: &service.CreateDownloadLinkRequest_NodeExecutionId{ + NodeExecutionId: &core.NodeExecutionIdentifier{}, + }, + ExpiresIn: durationpb.New(time.Hour), + }) + + st, ok := status.FromError(err) + assert.True(t, ok) + assert.Equal(t, codes.Internal, st.Code()) + assert.Equal(t, "failed to head object before signing url. Error: head fail", st.Message()) + }) + t.Run("use default ExpiresIn", func(t *testing.T) { + dataStore.ComposedProtobufStore.(*commonMocks.TestDataStore).HeadCb = func(ctx context.Context, ref storage.DataReference) (storage.Metadata, error) { + return testMetadata{exists: true}, nil + } + _, err = s.CreateDownloadLink(context.Background(), &service.CreateDownloadLinkRequest{ ArtifactType: service.ArtifactType_ARTIFACT_TYPE_DECK, Source: &service.CreateDownloadLinkRequest_NodeExecutionId{ From 765ce2e6587180bb71283750c6db79fbd6232a68 Mon Sep 17 00:00:00 2001 From: Iaroslav Ciupin Date: Wed, 14 Aug 2024 22:37:37 +0300 Subject: [PATCH 23/25] Unexpectedly deleted pod metrics * Count when we see unexpectedly terminated pods Signed-off-by: Andrew Dye --- .../controller/nodes/task/k8s/plugin_manager.go | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/flytepropeller/pkg/controller/nodes/task/k8s/plugin_manager.go b/flytepropeller/pkg/controller/nodes/task/k8s/plugin_manager.go index 42d3ad9b85..c9c9167146 100644 --- a/flytepropeller/pkg/controller/nodes/task/k8s/plugin_manager.go +++ b/flytepropeller/pkg/controller/nodes/task/k8s/plugin_manager.go @@ -5,6 +5,7 @@ import ( "fmt" "time" + "github.com/prometheus/client_golang/prometheus" "golang.org/x/time/rate" v1 "k8s.io/api/core/v1" k8serrors "k8s.io/apimachinery/pkg/api/errors" @@ -64,6 +65,7 @@ type PluginMetrics struct { GetCacheHit labeled.StopWatch GetAPILatency labeled.StopWatch ResourceDeleted labeled.Counter + TaskPodErrors *prometheus.CounterVec } func newPluginMetrics(s promutils.Scope) PluginMetrics { @@ -77,6 +79,8 @@ func newPluginMetrics(s promutils.Scope) PluginMetrics { time.Millisecond, s), ResourceDeleted: labeled.NewCounter("pods_deleted", "Counts how many times CheckTaskStatus is"+ " called with a deleted resource.", s), + TaskPodErrors: s.MustNewCounterVec("task_pod_errors", "Counts how many times task pods failed in given phase with given code", + "phase", "error_code"), } } @@ -350,14 +354,19 @@ func (e PluginManager) Handle(ctx context.Context, tCtx pluginsCore.TaskExecutio return transition, err } + phaseInfo := transition.Info() + if phaseInfo.Err() != nil { + e.metrics.TaskPodErrors.WithLabelValues(phaseInfo.Phase().String(), phaseInfo.Err().GetCode()).Inc() + } + // Add events since last update - version := transition.Info().Version() + version := phaseInfo.Version() lastEventUpdate := pluginState.LastEventUpdate if e.eventWatcher != nil && o != nil { nsName := k8stypes.NamespacedName{Namespace: o.GetNamespace(), Name: o.GetName()} recentEvents := e.eventWatcher.List(nsName, lastEventUpdate) if len(recentEvents) > 0 { - taskInfo := transition.Info().Info() + taskInfo := phaseInfo.Info() taskInfo.AdditionalReasons = make([]pluginsCore.ReasonInfo, 0, len(recentEvents)) for _, event := range recentEvents { taskInfo.AdditionalReasons = append(taskInfo.AdditionalReasons, @@ -373,9 +382,9 @@ func (e PluginManager) Handle(ctx context.Context, tCtx pluginsCore.TaskExecutio newPluginState := PluginState{ Phase: pluginPhase, K8sPluginState: k8s.PluginState{ - Phase: transition.Info().Phase(), + Phase: phaseInfo.Phase(), PhaseVersion: version, - Reason: transition.Info().Reason(), + Reason: phaseInfo.Reason(), }, LastEventUpdate: lastEventUpdate, } From 184676403649d458889d83f44b903869b1bbe762 Mon Sep 17 00:00:00 2001 From: Iaroslav Ciupin Date: Mon, 12 Aug 2024 17:37:21 +0300 Subject: [PATCH 24/25] Don't send inputURI for start-node * send empty `inputUri` for `start-node` in node execution event to flyteadmin and therefore, GetNodeExecutionData will not attempt to download non-existing inputUri as it was doing before this change. * add DB migration to clear `input_uri` in existing `node_executions` table for start nodes. Signed-off-by: Andrew Dye --- .../pkg/repositories/config/migrations.go | 12 ++++++-- .../pkg/controller/nodes/executor.go | 15 +++++++--- .../pkg/controller/nodes/executor_test.go | 8 +++++ .../pkg/controller/nodes/transformers.go | 30 ++++++++++--------- 4 files changed, 45 insertions(+), 20 deletions(-) diff --git a/flyteadmin/pkg/repositories/config/migrations.go b/flyteadmin/pkg/repositories/config/migrations.go index 27da97f29c..d48356fe01 100644 --- a/flyteadmin/pkg/repositories/config/migrations.go +++ b/flyteadmin/pkg/repositories/config/migrations.go @@ -1263,20 +1263,28 @@ var ContinuedMigrations = []*gormigrate.Migration{ return tx.Migrator().DropTable("execution_tags") }, }, - { ID: "2024-06-06-drop-execution_admin-tags", Migrate: func(tx *gorm.DB) error { return tx.Migrator().DropTable("execution_admin_tags") }, }, - { ID: "2024-06-06-drop-admin-tags", Migrate: func(tx *gorm.DB) error { return tx.Migrator().DropTable("admin_tags") }, }, + { + ID: "2024-08-08-remove-input-uri-for-start-nodes", + Migrate: func(db *gorm.DB) error { + return db.Exec("UPDATE node_executions SET input_uri = '' WHERE node_id = 'start-node'").Error + }, + Rollback: func(db *gorm.DB) error { + // can't rollback missing data + return nil + }, + }, } var m = append(LegacyMigrations, NoopMigrations...) diff --git a/flytepropeller/pkg/controller/nodes/executor.go b/flytepropeller/pkg/controller/nodes/executor.go index 2c3103e4ad..b25ad64fb6 100644 --- a/flytepropeller/pkg/controller/nodes/executor.go +++ b/flytepropeller/pkg/controller/nodes/executor.go @@ -1248,10 +1248,17 @@ func (c *nodeExecutor) handleQueuedOrRunningNode(ctx context.Context, nCtx inter targetEntity := common.GetTargetEntity(ctx, nCtx) - nev, err := ToNodeExecutionEvent(nCtx.NodeExecutionMetadata().GetNodeExecutionID(), - p, nCtx.InputReader().GetInputPath().String(), nCtx.NodeStatus(), nCtx.ExecutionContext().GetEventVersion(), - nCtx.ExecutionContext().GetParentInfo(), nCtx.Node(), c.clusterID, nCtx.NodeStateReader().GetDynamicNodeState().Phase, - c.eventConfig, targetEntity) + nev, err := ToNodeExecutionEvent( + nCtx.NodeExecutionMetadata().GetNodeExecutionID(), + p, + nCtx.InputReader().GetInputPath().String(), + nCtx.NodeStatus(), + nCtx.ExecutionContext().GetEventVersion(), + nCtx.ExecutionContext().GetParentInfo(), nCtx.Node(), + c.clusterID, + nCtx.NodeStateReader().GetDynamicNodeState().Phase, + c.eventConfig, + targetEntity) if err != nil { return interfaces.NodeStatusUndefined, errors.Wrapf(errors.IllegalStateError, nCtx.NodeID(), err, "could not convert phase info to event") } diff --git a/flytepropeller/pkg/controller/nodes/executor_test.go b/flytepropeller/pkg/controller/nodes/executor_test.go index 7fc4c05992..35ab105623 100644 --- a/flytepropeller/pkg/controller/nodes/executor_test.go +++ b/flytepropeller/pkg/controller/nodes/executor_test.go @@ -1723,6 +1723,7 @@ func TestNodeExecutor_FinalizeHandler(t *testing.T) { assert.NoError(t, exec.FinalizeHandler(ctx, nil, nil, nl, n)) }) } + func TestNodeExecutionEventStartNode(t *testing.T) { execID := &core.WorkflowExecutionIdentifier{ Name: "e1", @@ -1763,9 +1764,11 @@ func TestNodeExecutionEventStartNode(t *testing.T) { ns.OnGetParentTaskID().Return(tID) ns.OnGetOutputDirMatch(mock.Anything).Return("dummy://dummyOutUrl") ns.OnGetDynamicNodeStatus().Return(&v1alpha1.DynamicNodeStatus{}) + ev, err := ToNodeExecutionEvent(nID, p, "reference", ns, v1alpha1.EventVersion0, parentInfo, n, testClusterID, v1alpha1.DynamicNodePhaseNone, &config.EventConfig{ RawOutputPolicy: config.RawOutputPolicyReference, }, subWfID) + assert.NoError(t, err) assert.Equal(t, "start-node", ev.Id.NodeId) assert.Equal(t, execID, ev.Id.ExecutionId) @@ -1778,6 +1781,7 @@ func TestNodeExecutionEventStartNode(t *testing.T) { ev.OutputResult.(*event.NodeExecutionEvent_OutputUri).OutputUri) assert.Equal(t, ev.ProducerId, testClusterID) assert.Equal(t, subWfID, ev.GetTargetEntity()) + assert.Nil(t, ev.InputValue) } func TestNodeExecutionEventV0(t *testing.T) { @@ -1821,6 +1825,7 @@ func TestNodeExecutionEventV0(t *testing.T) { assert.Empty(t, ev.NodeName) assert.Empty(t, ev.RetryGroup) assert.Empty(t, ev.TargetEntity) + assert.Equal(t, "reference", ev.GetInputUri()) } func TestNodeExecutionEventV1(t *testing.T) { @@ -1859,9 +1864,11 @@ func TestNodeExecutionEventV1(t *testing.T) { ns.OnGetPhase().Return(v1alpha1.NodePhaseNotYetStarted) nl.OnGetNodeExecutionStatusMatch(mock.Anything, id).Return(ns) ns.OnGetParentTaskID().Return(tID) + eventOpt, err := ToNodeExecutionEvent(nID, p, "reference", ns, v1alpha1.EventVersion1, parentInfo, n, testClusterID, v1alpha1.DynamicNodePhaseNone, &config.EventConfig{ RawOutputPolicy: config.RawOutputPolicyInline, }, nil) + assert.NoError(t, err) assert.Equal(t, "np1-2-n1", eventOpt.Id.NodeId) assert.Equal(t, execID, eventOpt.Id.ExecutionId) @@ -1875,6 +1882,7 @@ func TestNodeExecutionEventV1(t *testing.T) { assert.Equal(t, "2", eventOpt.RetryGroup) assert.True(t, proto.Equal(eventOpt.GetInputData(), inputs)) assert.Empty(t, eventOpt.TargetEntity) + assert.Equal(t, inputs, eventOpt.GetInputData()) } func TestNodeExecutor_RecursiveNodeHandler_ParallelismLimit(t *testing.T) { diff --git a/flytepropeller/pkg/controller/nodes/transformers.go b/flytepropeller/pkg/controller/nodes/transformers.go index c9f7d5fc76..a252d17344 100644 --- a/flytepropeller/pkg/controller/nodes/transformers.go +++ b/flytepropeller/pkg/controller/nodes/transformers.go @@ -76,7 +76,8 @@ func ToNodeExecEventPhase(p handler.EPhase) core.NodeExecution_Phase { } } -func ToNodeExecutionEvent(nodeExecID *core.NodeExecutionIdentifier, +func ToNodeExecutionEvent( + nodeExecID *core.NodeExecutionIdentifier, info handler.PhaseInfo, inputPath string, status v1alpha1.ExecutableNodeStatus, @@ -109,9 +110,11 @@ func ToNodeExecutionEvent(nodeExecID *core.NodeExecutionIdentifier, dynamicChain = true } + eInfo := info.GetInfo() var nev *event.NodeExecutionEvent - // Start node is special case where the Inputs and Outputs are the same and hence here we copy the Output file + // Start node is special case where the Outputs are the same and hence here we copy the Output file // into the OutputResult and in admin we copy it over into input as well. + // Start node doesn't have inputs. if nodeExecID.NodeId == v1alpha1.StartNodeID { outputsFile := v1alpha1.GetOutputsFile(status.GetOutputDir()) nev = &event.NodeExecutionEvent{ @@ -139,6 +142,17 @@ func ToNodeExecutionEvent(nodeExecID *core.NodeExecutionIdentifier, TargetEntity: targetEntity, IsInDynamicChain: dynamicChain, } + if eventConfig.RawOutputPolicy == config.RawOutputPolicyInline { + if eInfo != nil { + nev.InputValue = &event.NodeExecutionEvent_InputData{ + InputData: eInfo.Inputs, + } + } + } else { + nev.InputValue = &event.NodeExecutionEvent_InputUri{ + InputUri: inputPath, + } + } } if eventVersion == v1alpha1.EventVersion0 && status.GetParentTaskID() != nil { @@ -163,7 +177,6 @@ func ToNodeExecutionEvent(nodeExecID *core.NodeExecutionIdentifier, nev.NodeName = node.GetName() } - eInfo := info.GetInfo() if eInfo != nil { if eInfo.WorkflowNodeInfo != nil { v := ToNodeExecWorkflowNodeMetadata(eInfo.WorkflowNodeInfo) @@ -201,17 +214,6 @@ func ToNodeExecutionEvent(nodeExecID *core.NodeExecutionIdentifier, nev.IsParent = true } } - if eventConfig.RawOutputPolicy == config.RawOutputPolicyInline { - if eInfo != nil { - nev.InputValue = &event.NodeExecutionEvent_InputData{ - InputData: eInfo.Inputs, - } - } - } else { - nev.InputValue = &event.NodeExecutionEvent_InputUri{ - InputUri: inputPath, - } - } return nev, nil } From 895344debb3be46411881bdb0237badc9085149e Mon Sep 17 00:00:00 2001 From: Iaroslav Ciupin Date: Wed, 10 Jul 2024 11:58:12 +0300 Subject: [PATCH 25/25] Fix cluster pool assignment validation Signed-off-by: Andrew Dye --- .../pkg/manager/impl/execution_manager.go | 41 +- .../manager/impl/execution_manager_test.go | 112 ++++- flyteadmin/pkg/manager/interfaces/resource.go | 2 + .../pkg/manager/mocks/resource_interface.go | 469 ++++++++++++++++++ 4 files changed, 599 insertions(+), 25 deletions(-) create mode 100644 flyteadmin/pkg/manager/mocks/resource_interface.go diff --git a/flyteadmin/pkg/manager/impl/execution_manager.go b/flyteadmin/pkg/manager/impl/execution_manager.go index 7c64ccbc3c..27acf152ec 100644 --- a/flyteadmin/pkg/manager/impl/execution_manager.go +++ b/flyteadmin/pkg/manager/impl/execution_manager.go @@ -403,15 +403,34 @@ func (m *ExecutionManager) getExecutionConfig(ctx context.Context, request *admi return workflowExecConfig, nil } -func (m *ExecutionManager) getClusterAssignment(ctx context.Context, request *admin.ExecutionCreateRequest) ( - *admin.ClusterAssignment, error) { - if request.Spec.ClusterAssignment != nil { - return request.Spec.ClusterAssignment, nil +func (m *ExecutionManager) getClusterAssignment(ctx context.Context, req *admin.ExecutionCreateRequest) (*admin.ClusterAssignment, error) { + storedAssignment, err := m.fetchClusterAssignment(ctx, req.Project, req.Domain) + if err != nil { + return nil, err + } + + reqAssignment := req.GetSpec().GetClusterAssignment() + reqPool := reqAssignment.GetClusterPoolName() + storedPool := storedAssignment.GetClusterPoolName() + if reqPool == "" { + return storedAssignment, nil + } + + if storedPool == "" { + return reqAssignment, nil } + if reqPool != storedPool { + return nil, errors.NewFlyteAdminErrorf(codes.InvalidArgument, "execution with project %q and domain %q cannot run on cluster pool %q, because its configured to run on pool %q", req.Project, req.Domain, reqPool, storedPool) + } + + return storedAssignment, nil +} + +func (m *ExecutionManager) fetchClusterAssignment(ctx context.Context, project, domain string) (*admin.ClusterAssignment, error) { resource, err := m.resourceManager.GetResource(ctx, interfaces.ResourceRequest{ - Project: request.Project, - Domain: request.Domain, + Project: project, + Domain: domain, ResourceType: admin.MatchableResource_CLUSTER_ASSIGNMENT, }) if err != nil && !errors.IsDoesNotExistError(err) { @@ -421,11 +440,13 @@ func (m *ExecutionManager) getClusterAssignment(ctx context.Context, request *ad if resource != nil && resource.Attributes.GetClusterAssignment() != nil { return resource.Attributes.GetClusterAssignment(), nil } - clusterPoolAssignment := m.config.ClusterPoolAssignmentConfiguration().GetClusterPoolAssignments()[request.GetDomain()] - return &admin.ClusterAssignment{ - ClusterPoolName: clusterPoolAssignment.Pool, - }, nil + var clusterAssignment *admin.ClusterAssignment + domainAssignment := m.config.ClusterPoolAssignmentConfiguration().GetClusterPoolAssignments()[domain] + if domainAssignment.Pool != "" { + clusterAssignment = &admin.ClusterAssignment{ClusterPoolName: domainAssignment.Pool} + } + return clusterAssignment, nil } func (m *ExecutionManager) launchSingleTaskExecution( diff --git a/flyteadmin/pkg/manager/impl/execution_manager_test.go b/flyteadmin/pkg/manager/impl/execution_manager_test.go index 93d327bd53..5e874a4589 100644 --- a/flyteadmin/pkg/manager/impl/execution_manager_test.go +++ b/flyteadmin/pkg/manager/impl/execution_manager_test.go @@ -304,8 +304,7 @@ func TestCreateExecution(t *testing.T) { }} repository.ProjectRepo().(*repositoryMocks.MockProjectRepo).GetFunction = func( ctx context.Context, projectID string) (models.Project, error) { - return transformers.CreateProjectModel(&admin.Project{ - Labels: &labels}), nil + return transformers.CreateProjectModel(&admin.Project{Labels: &labels}), nil } clusterAssignment := admin.ClusterAssignment{ClusterPoolName: "gpu"} @@ -382,8 +381,6 @@ func TestCreateExecution(t *testing.T) { mockConfig := getMockExecutionsConfigProvider() mockConfig.(*runtimeMocks.MockConfigurationProvider).AddQualityOfServiceConfiguration(qosProvider) - - execManager := NewExecutionManager(repository, r, mockConfig, getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) request := testutils.GetExecutionRequest() request.Spec.Metadata = &admin.ExecutionMetadata{ Principal: "unused - populated from authenticated context", @@ -392,16 +389,18 @@ func TestCreateExecution(t *testing.T) { request.Spec.ClusterAssignment = &clusterAssignment request.Spec.ExecutionClusterLabel = &admin.ExecutionClusterLabel{Value: executionClusterLabel} + execManager := NewExecutionManager(repository, r, mockConfig, getMockStorageForExecTest(context.Background()), mockScope.NewTestScope(), mockScope.NewTestScope(), &mockPublisher, mockExecutionRemoteURL, nil, nil, &mockPublisher, nil, &eventWriterMocks.WorkflowExecutionEventWriter{}) + identity, err := auth.NewIdentityContext("", principal, "", time.Now(), sets.NewString(), nil, nil) assert.NoError(t, err) ctx := identity.WithContext(context.Background()) response, err := execManager.CreateExecution(ctx, request, requestedAt) - assert.Nil(t, err) + assert.NoError(t, err) expectedResponse := &admin.ExecutionCreateResponse{ Id: &executionIdentifier, } - assert.Nil(t, err) + assert.NoError(t, err) assert.True(t, proto.Equal(expectedResponse.Id, response.Id)) // TODO: Check for offloaded inputs @@ -632,7 +631,6 @@ func TestCreateExecutionInCompatibleInputs(t *testing.T) { } func TestCreateExecutionPropellerFailure(t *testing.T) { - clusterAssignment := admin.ClusterAssignment{ClusterPoolName: "gpu"} repository := getMockRepositoryForExecTest() setDefaultLpCallbackForExecTest(repository) expectedErr := flyteAdminErrors.NewFlyteAdminErrorf(codes.Internal, "ABC") @@ -666,7 +664,6 @@ func TestCreateExecutionPropellerFailure(t *testing.T) { Principal: "unused - populated from authenticated context", } request.Spec.RawOutputDataConfig = &admin.RawOutputDataConfig{OutputLocationPrefix: rawOutput} - request.Spec.ClusterAssignment = &clusterAssignment identity, err := auth.NewIdentityContext("", principal, "", time.Now(), sets.NewString(), nil, nil) assert.NoError(t, err) @@ -5467,8 +5464,32 @@ func TestGetClusterAssignment(t *testing.T) { assert.NoError(t, err) assert.True(t, proto.Equal(ca, &clusterAssignment)) }) - t.Run("value from request", func(t *testing.T) { - reqClusterAssignment := admin.ClusterAssignment{ClusterPoolName: "swimming-pool"} + t.Run("value from config", func(t *testing.T) { + customCP := "my_cp" + clusterPoolAsstProvider := &runtimeIFaceMocks.ClusterPoolAssignmentConfiguration{} + clusterPoolAsstProvider.OnGetClusterPoolAssignments().Return(runtimeInterfaces.ClusterPoolAssignments{ + workflowIdentifier.GetDomain(): runtimeInterfaces.ClusterPoolAssignment{ + Pool: customCP, + }, + }) + mockConfig := getMockExecutionsConfigProvider() + mockConfig.(*runtimeMocks.MockConfigurationProvider).AddClusterPoolAssignmentConfiguration(clusterPoolAsstProvider) + + executionManager := ExecutionManager{ + resourceManager: &managerMocks.MockResourceManager{}, + config: mockConfig, + } + + ca, err := executionManager.getClusterAssignment(context.TODO(), &admin.ExecutionCreateRequest{ + Project: workflowIdentifier.Project, + Domain: workflowIdentifier.Domain, + Spec: &admin.ExecutionSpec{}, + }) + assert.NoError(t, err) + assert.Equal(t, customCP, ca.GetClusterPoolName()) + }) + t.Run("value from request matches value from config", func(t *testing.T) { + reqClusterAssignment := admin.ClusterAssignment{ClusterPoolName: "gpu"} ca, err := executionManager.getClusterAssignment(context.TODO(), &admin.ExecutionCreateRequest{ Project: workflowIdentifier.Project, Domain: workflowIdentifier.Domain, @@ -5479,12 +5500,30 @@ func TestGetClusterAssignment(t *testing.T) { assert.NoError(t, err) assert.True(t, proto.Equal(ca, &reqClusterAssignment)) }) - t.Run("value from config", func(t *testing.T) { - customCP := "my_cp" + t.Run("no value in DB nor in config, takes value from request", func(t *testing.T) { + mockConfig := getMockExecutionsConfigProvider() + + executionManager := ExecutionManager{ + resourceManager: &managerMocks.MockResourceManager{}, + config: mockConfig, + } + + reqClusterAssignment := admin.ClusterAssignment{ClusterPoolName: "gpu"} + ca, err := executionManager.getClusterAssignment(context.TODO(), &admin.ExecutionCreateRequest{ + Project: workflowIdentifier.Project, + Domain: workflowIdentifier.Domain, + Spec: &admin.ExecutionSpec{ + ClusterAssignment: &reqClusterAssignment, + }, + }) + assert.NoError(t, err) + assert.True(t, proto.Equal(ca, &reqClusterAssignment)) + }) + t.Run("empty value in DB, takes value from request", func(t *testing.T) { clusterPoolAsstProvider := &runtimeIFaceMocks.ClusterPoolAssignmentConfiguration{} clusterPoolAsstProvider.OnGetClusterPoolAssignments().Return(runtimeInterfaces.ClusterPoolAssignments{ workflowIdentifier.GetDomain(): runtimeInterfaces.ClusterPoolAssignment{ - Pool: customCP, + Pool: "", }, }) mockConfig := getMockExecutionsConfigProvider() @@ -5495,13 +5534,56 @@ func TestGetClusterAssignment(t *testing.T) { config: mockConfig, } + reqClusterAssignment := admin.ClusterAssignment{ClusterPoolName: "gpu"} ca, err := executionManager.getClusterAssignment(context.TODO(), &admin.ExecutionCreateRequest{ Project: workflowIdentifier.Project, Domain: workflowIdentifier.Domain, - Spec: &admin.ExecutionSpec{}, + Spec: &admin.ExecutionSpec{ + ClusterAssignment: &reqClusterAssignment, + }, }) assert.NoError(t, err) - assert.Equal(t, customCP, ca.GetClusterPoolName()) + assert.True(t, proto.Equal(ca, &reqClusterAssignment)) + }) + t.Run("value from request doesn't match value from config", func(t *testing.T) { + reqClusterAssignment := admin.ClusterAssignment{ClusterPoolName: "swimming-pool"} + _, err := executionManager.getClusterAssignment(context.TODO(), &admin.ExecutionCreateRequest{ + Project: workflowIdentifier.Project, + Domain: workflowIdentifier.Domain, + Spec: &admin.ExecutionSpec{ + ClusterAssignment: &reqClusterAssignment, + }, + }) + st, ok := status.FromError(err) + assert.True(t, ok) + assert.Equal(t, codes.InvalidArgument, st.Code()) + assert.Equal(t, `execution with project "project" and domain "domain" cannot run on cluster pool "swimming-pool", because its configured to run on pool "gpu"`, st.Message()) + }) + t.Run("db error", func(t *testing.T) { + expected := errors.New("fail db") + resourceManager.GetResourceFunc = func(ctx context.Context, + request managerInterfaces.ResourceRequest) (*managerInterfaces.ResourceResponse, error) { + assert.EqualValues(t, request, managerInterfaces.ResourceRequest{ + Project: workflowIdentifier.Project, + Domain: workflowIdentifier.Domain, + ResourceType: admin.MatchableResource_CLUSTER_ASSIGNMENT, + }) + return &managerInterfaces.ResourceResponse{ + Attributes: &admin.MatchingAttributes{ + Target: &admin.MatchingAttributes_ClusterAssignment{ + ClusterAssignment: &clusterAssignment, + }, + }, + }, expected + } + + _, err := executionManager.getClusterAssignment(context.TODO(), &admin.ExecutionCreateRequest{ + Project: workflowIdentifier.Project, + Domain: workflowIdentifier.Domain, + Spec: &admin.ExecutionSpec{}, + }) + + assert.Equal(t, expected, err) }) } diff --git a/flyteadmin/pkg/manager/interfaces/resource.go b/flyteadmin/pkg/manager/interfaces/resource.go index 928a910d6c..3d586a59c9 100644 --- a/flyteadmin/pkg/manager/interfaces/resource.go +++ b/flyteadmin/pkg/manager/interfaces/resource.go @@ -6,6 +6,8 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" ) +//go:generate mockery -name ResourceInterface -output=../mocks -case=underscore + // ResourceInterface manages project, domain and workflow -specific attributes. type ResourceInterface interface { ListAll(ctx context.Context, request *admin.ListMatchableAttributesRequest) ( diff --git a/flyteadmin/pkg/manager/mocks/resource_interface.go b/flyteadmin/pkg/manager/mocks/resource_interface.go new file mode 100644 index 0000000000..c1b416eb9d --- /dev/null +++ b/flyteadmin/pkg/manager/mocks/resource_interface.go @@ -0,0 +1,469 @@ +// Code generated by mockery v1.0.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + admin "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" + + interfaces "github.com/flyteorg/flyte/flyteadmin/pkg/manager/interfaces" + + mock "github.com/stretchr/testify/mock" +) + +// ResourceInterface is an autogenerated mock type for the ResourceInterface type +type ResourceInterface struct { + mock.Mock +} + +type ResourceInterface_DeleteProjectAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_DeleteProjectAttributes) Return(_a0 *admin.ProjectAttributesDeleteResponse, _a1 error) *ResourceInterface_DeleteProjectAttributes { + return &ResourceInterface_DeleteProjectAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnDeleteProjectAttributes(ctx context.Context, request *admin.ProjectAttributesDeleteRequest) *ResourceInterface_DeleteProjectAttributes { + c_call := _m.On("DeleteProjectAttributes", ctx, request) + return &ResourceInterface_DeleteProjectAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnDeleteProjectAttributesMatch(matchers ...interface{}) *ResourceInterface_DeleteProjectAttributes { + c_call := _m.On("DeleteProjectAttributes", matchers...) + return &ResourceInterface_DeleteProjectAttributes{Call: c_call} +} + +// DeleteProjectAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) DeleteProjectAttributes(ctx context.Context, request *admin.ProjectAttributesDeleteRequest) (*admin.ProjectAttributesDeleteResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.ProjectAttributesDeleteResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.ProjectAttributesDeleteRequest) *admin.ProjectAttributesDeleteResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.ProjectAttributesDeleteResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.ProjectAttributesDeleteRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_DeleteProjectDomainAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_DeleteProjectDomainAttributes) Return(_a0 *admin.ProjectDomainAttributesDeleteResponse, _a1 error) *ResourceInterface_DeleteProjectDomainAttributes { + return &ResourceInterface_DeleteProjectDomainAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnDeleteProjectDomainAttributes(ctx context.Context, request *admin.ProjectDomainAttributesDeleteRequest) *ResourceInterface_DeleteProjectDomainAttributes { + c_call := _m.On("DeleteProjectDomainAttributes", ctx, request) + return &ResourceInterface_DeleteProjectDomainAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnDeleteProjectDomainAttributesMatch(matchers ...interface{}) *ResourceInterface_DeleteProjectDomainAttributes { + c_call := _m.On("DeleteProjectDomainAttributes", matchers...) + return &ResourceInterface_DeleteProjectDomainAttributes{Call: c_call} +} + +// DeleteProjectDomainAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) DeleteProjectDomainAttributes(ctx context.Context, request *admin.ProjectDomainAttributesDeleteRequest) (*admin.ProjectDomainAttributesDeleteResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.ProjectDomainAttributesDeleteResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.ProjectDomainAttributesDeleteRequest) *admin.ProjectDomainAttributesDeleteResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.ProjectDomainAttributesDeleteResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.ProjectDomainAttributesDeleteRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_DeleteWorkflowAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_DeleteWorkflowAttributes) Return(_a0 *admin.WorkflowAttributesDeleteResponse, _a1 error) *ResourceInterface_DeleteWorkflowAttributes { + return &ResourceInterface_DeleteWorkflowAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnDeleteWorkflowAttributes(ctx context.Context, request *admin.WorkflowAttributesDeleteRequest) *ResourceInterface_DeleteWorkflowAttributes { + c_call := _m.On("DeleteWorkflowAttributes", ctx, request) + return &ResourceInterface_DeleteWorkflowAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnDeleteWorkflowAttributesMatch(matchers ...interface{}) *ResourceInterface_DeleteWorkflowAttributes { + c_call := _m.On("DeleteWorkflowAttributes", matchers...) + return &ResourceInterface_DeleteWorkflowAttributes{Call: c_call} +} + +// DeleteWorkflowAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) DeleteWorkflowAttributes(ctx context.Context, request *admin.WorkflowAttributesDeleteRequest) (*admin.WorkflowAttributesDeleteResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.WorkflowAttributesDeleteResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.WorkflowAttributesDeleteRequest) *admin.WorkflowAttributesDeleteResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.WorkflowAttributesDeleteResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.WorkflowAttributesDeleteRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_GetProjectAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_GetProjectAttributes) Return(_a0 *admin.ProjectAttributesGetResponse, _a1 error) *ResourceInterface_GetProjectAttributes { + return &ResourceInterface_GetProjectAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnGetProjectAttributes(ctx context.Context, request *admin.ProjectAttributesGetRequest) *ResourceInterface_GetProjectAttributes { + c_call := _m.On("GetProjectAttributes", ctx, request) + return &ResourceInterface_GetProjectAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnGetProjectAttributesMatch(matchers ...interface{}) *ResourceInterface_GetProjectAttributes { + c_call := _m.On("GetProjectAttributes", matchers...) + return &ResourceInterface_GetProjectAttributes{Call: c_call} +} + +// GetProjectAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) GetProjectAttributes(ctx context.Context, request *admin.ProjectAttributesGetRequest) (*admin.ProjectAttributesGetResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.ProjectAttributesGetResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.ProjectAttributesGetRequest) *admin.ProjectAttributesGetResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.ProjectAttributesGetResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.ProjectAttributesGetRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_GetProjectDomainAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_GetProjectDomainAttributes) Return(_a0 *admin.ProjectDomainAttributesGetResponse, _a1 error) *ResourceInterface_GetProjectDomainAttributes { + return &ResourceInterface_GetProjectDomainAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnGetProjectDomainAttributes(ctx context.Context, request *admin.ProjectDomainAttributesGetRequest) *ResourceInterface_GetProjectDomainAttributes { + c_call := _m.On("GetProjectDomainAttributes", ctx, request) + return &ResourceInterface_GetProjectDomainAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnGetProjectDomainAttributesMatch(matchers ...interface{}) *ResourceInterface_GetProjectDomainAttributes { + c_call := _m.On("GetProjectDomainAttributes", matchers...) + return &ResourceInterface_GetProjectDomainAttributes{Call: c_call} +} + +// GetProjectDomainAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) GetProjectDomainAttributes(ctx context.Context, request *admin.ProjectDomainAttributesGetRequest) (*admin.ProjectDomainAttributesGetResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.ProjectDomainAttributesGetResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.ProjectDomainAttributesGetRequest) *admin.ProjectDomainAttributesGetResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.ProjectDomainAttributesGetResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.ProjectDomainAttributesGetRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_GetResource struct { + *mock.Call +} + +func (_m ResourceInterface_GetResource) Return(_a0 *interfaces.ResourceResponse, _a1 error) *ResourceInterface_GetResource { + return &ResourceInterface_GetResource{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnGetResource(ctx context.Context, request interfaces.ResourceRequest) *ResourceInterface_GetResource { + c_call := _m.On("GetResource", ctx, request) + return &ResourceInterface_GetResource{Call: c_call} +} + +func (_m *ResourceInterface) OnGetResourceMatch(matchers ...interface{}) *ResourceInterface_GetResource { + c_call := _m.On("GetResource", matchers...) + return &ResourceInterface_GetResource{Call: c_call} +} + +// GetResource provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) GetResource(ctx context.Context, request interfaces.ResourceRequest) (*interfaces.ResourceResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *interfaces.ResourceResponse + if rf, ok := ret.Get(0).(func(context.Context, interfaces.ResourceRequest) *interfaces.ResourceResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*interfaces.ResourceResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, interfaces.ResourceRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_GetWorkflowAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_GetWorkflowAttributes) Return(_a0 *admin.WorkflowAttributesGetResponse, _a1 error) *ResourceInterface_GetWorkflowAttributes { + return &ResourceInterface_GetWorkflowAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnGetWorkflowAttributes(ctx context.Context, request *admin.WorkflowAttributesGetRequest) *ResourceInterface_GetWorkflowAttributes { + c_call := _m.On("GetWorkflowAttributes", ctx, request) + return &ResourceInterface_GetWorkflowAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnGetWorkflowAttributesMatch(matchers ...interface{}) *ResourceInterface_GetWorkflowAttributes { + c_call := _m.On("GetWorkflowAttributes", matchers...) + return &ResourceInterface_GetWorkflowAttributes{Call: c_call} +} + +// GetWorkflowAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) GetWorkflowAttributes(ctx context.Context, request *admin.WorkflowAttributesGetRequest) (*admin.WorkflowAttributesGetResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.WorkflowAttributesGetResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.WorkflowAttributesGetRequest) *admin.WorkflowAttributesGetResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.WorkflowAttributesGetResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.WorkflowAttributesGetRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_ListAll struct { + *mock.Call +} + +func (_m ResourceInterface_ListAll) Return(_a0 *admin.ListMatchableAttributesResponse, _a1 error) *ResourceInterface_ListAll { + return &ResourceInterface_ListAll{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnListAll(ctx context.Context, request *admin.ListMatchableAttributesRequest) *ResourceInterface_ListAll { + c_call := _m.On("ListAll", ctx, request) + return &ResourceInterface_ListAll{Call: c_call} +} + +func (_m *ResourceInterface) OnListAllMatch(matchers ...interface{}) *ResourceInterface_ListAll { + c_call := _m.On("ListAll", matchers...) + return &ResourceInterface_ListAll{Call: c_call} +} + +// ListAll provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) ListAll(ctx context.Context, request *admin.ListMatchableAttributesRequest) (*admin.ListMatchableAttributesResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.ListMatchableAttributesResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.ListMatchableAttributesRequest) *admin.ListMatchableAttributesResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.ListMatchableAttributesResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.ListMatchableAttributesRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_UpdateProjectAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_UpdateProjectAttributes) Return(_a0 *admin.ProjectAttributesUpdateResponse, _a1 error) *ResourceInterface_UpdateProjectAttributes { + return &ResourceInterface_UpdateProjectAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnUpdateProjectAttributes(ctx context.Context, request *admin.ProjectAttributesUpdateRequest) *ResourceInterface_UpdateProjectAttributes { + c_call := _m.On("UpdateProjectAttributes", ctx, request) + return &ResourceInterface_UpdateProjectAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnUpdateProjectAttributesMatch(matchers ...interface{}) *ResourceInterface_UpdateProjectAttributes { + c_call := _m.On("UpdateProjectAttributes", matchers...) + return &ResourceInterface_UpdateProjectAttributes{Call: c_call} +} + +// UpdateProjectAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) UpdateProjectAttributes(ctx context.Context, request *admin.ProjectAttributesUpdateRequest) (*admin.ProjectAttributesUpdateResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.ProjectAttributesUpdateResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.ProjectAttributesUpdateRequest) *admin.ProjectAttributesUpdateResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.ProjectAttributesUpdateResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.ProjectAttributesUpdateRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_UpdateProjectDomainAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_UpdateProjectDomainAttributes) Return(_a0 *admin.ProjectDomainAttributesUpdateResponse, _a1 error) *ResourceInterface_UpdateProjectDomainAttributes { + return &ResourceInterface_UpdateProjectDomainAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnUpdateProjectDomainAttributes(ctx context.Context, request *admin.ProjectDomainAttributesUpdateRequest) *ResourceInterface_UpdateProjectDomainAttributes { + c_call := _m.On("UpdateProjectDomainAttributes", ctx, request) + return &ResourceInterface_UpdateProjectDomainAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnUpdateProjectDomainAttributesMatch(matchers ...interface{}) *ResourceInterface_UpdateProjectDomainAttributes { + c_call := _m.On("UpdateProjectDomainAttributes", matchers...) + return &ResourceInterface_UpdateProjectDomainAttributes{Call: c_call} +} + +// UpdateProjectDomainAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) UpdateProjectDomainAttributes(ctx context.Context, request *admin.ProjectDomainAttributesUpdateRequest) (*admin.ProjectDomainAttributesUpdateResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.ProjectDomainAttributesUpdateResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.ProjectDomainAttributesUpdateRequest) *admin.ProjectDomainAttributesUpdateResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.ProjectDomainAttributesUpdateResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.ProjectDomainAttributesUpdateRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ResourceInterface_UpdateWorkflowAttributes struct { + *mock.Call +} + +func (_m ResourceInterface_UpdateWorkflowAttributes) Return(_a0 *admin.WorkflowAttributesUpdateResponse, _a1 error) *ResourceInterface_UpdateWorkflowAttributes { + return &ResourceInterface_UpdateWorkflowAttributes{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ResourceInterface) OnUpdateWorkflowAttributes(ctx context.Context, request *admin.WorkflowAttributesUpdateRequest) *ResourceInterface_UpdateWorkflowAttributes { + c_call := _m.On("UpdateWorkflowAttributes", ctx, request) + return &ResourceInterface_UpdateWorkflowAttributes{Call: c_call} +} + +func (_m *ResourceInterface) OnUpdateWorkflowAttributesMatch(matchers ...interface{}) *ResourceInterface_UpdateWorkflowAttributes { + c_call := _m.On("UpdateWorkflowAttributes", matchers...) + return &ResourceInterface_UpdateWorkflowAttributes{Call: c_call} +} + +// UpdateWorkflowAttributes provides a mock function with given fields: ctx, request +func (_m *ResourceInterface) UpdateWorkflowAttributes(ctx context.Context, request *admin.WorkflowAttributesUpdateRequest) (*admin.WorkflowAttributesUpdateResponse, error) { + ret := _m.Called(ctx, request) + + var r0 *admin.WorkflowAttributesUpdateResponse + if rf, ok := ret.Get(0).(func(context.Context, *admin.WorkflowAttributesUpdateRequest) *admin.WorkflowAttributesUpdateResponse); ok { + r0 = rf(ctx, request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*admin.WorkflowAttributesUpdateResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *admin.WorkflowAttributesUpdateRequest) error); ok { + r1 = rf(ctx, request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +}