From de5c1eea1f88a3ce1d8738d46a16464d1220a3e7 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Thu, 28 Nov 2024 13:02:47 -0800 Subject: [PATCH 01/21] add specific check for SDK GetWorkflowExecutionOptions path --- tests/deployment_test.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/tests/deployment_test.go b/tests/deployment_test.go index 4ef4e8e813e..7d1aa9820e1 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -486,6 +486,22 @@ func (s *DeploymentSuite) checkDeploymentReachability( }, 5*time.Second, 50*time.Millisecond) } +// SDK will have a GetWorkflowExecutionOptions method that sends an empty mask and a default +// WorkflowExecutionOptions and expects to read the workflow execution's existing options with no write +func (s *DeploymentSuite) checkSDKGetWorkflowExecutionOptions(ctx context.Context, + wf *commonpb.WorkflowExecution, + expectedOpts *workflowpb.WorkflowExecutionOptions, +) { + getResp, err := s.FrontendClient().UpdateWorkflowExecutionOptions(ctx, &workflowservice.UpdateWorkflowExecutionOptionsRequest{ + Namespace: s.Namespace(), + WorkflowExecution: wf, + WorkflowExecutionOptions: &workflowpb.WorkflowExecutionOptions{}, + UpdateMask: &fieldmaskpb.FieldMask{Paths: []string{}}, + }) + s.NoError(err) + s.True(proto.Equal(getResp.GetWorkflowExecutionOptions(), expectedOpts)) +} + func (s *DeploymentSuite) createDeploymentAndWaitForExist( deployment *deploymentpb.Deployment, tq *taskqueuepb.TaskQueue, @@ -539,6 +555,7 @@ func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_SetUnpinnedThenUnse s.NoError(err) s.True(proto.Equal(updateResp.GetWorkflowExecutionOptions(), unpinnedOpts)) s.checkDescribeWorkflowAfterOverride(ctx, unversionedWFExec, unpinnedOpts.GetVersioningOverride()) + s.checkSDKGetWorkflowExecutionOptions(ctx, unversionedWFExec, unpinnedOpts) // 2. Unset using empty update opts with mutation mask --> describe workflow shows no more override updateResp, err = s.FrontendClient().UpdateWorkflowExecutionOptions(ctx, &workflowservice.UpdateWorkflowExecutionOptionsRequest{ @@ -594,6 +611,7 @@ func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_SetPinnedThenUnset( s.True(proto.Equal(updateResp.GetWorkflowExecutionOptions(), pinnedOpts)) s.checkDescribeWorkflowAfterOverride(ctx, unversionedWFExec, pinnedOpts.GetVersioningOverride()) s.checkDeploymentReachability(ctx, workerDeployment, enumspb.DEPLOYMENT_REACHABILITY_REACHABLE) + s.checkSDKGetWorkflowExecutionOptions(ctx, unversionedWFExec, pinnedOpts) // 2. Unset with empty update opts with mutation mask --> describe workflow shows no more override + deployment is unreachable updateResp, err = s.FrontendClient().UpdateWorkflowExecutionOptions(ctx, &workflowservice.UpdateWorkflowExecutionOptionsRequest{ @@ -645,6 +663,7 @@ func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_EmptyFields() { s.NoError(err) s.True(proto.Equal(updateResp.GetWorkflowExecutionOptions(), &workflowpb.WorkflowExecutionOptions{})) s.checkDescribeWorkflowAfterOverride(ctx, unversionedWFExec, nil) + s.checkSDKGetWorkflowExecutionOptions(ctx, unversionedWFExec, &workflowpb.WorkflowExecutionOptions{}) } func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_SetPinnedSetPinned() { From 8663f0ac11c95ae7d7b0b9bbe2643fca7e732edc Mon Sep 17 00:00:00 2001 From: ShahabT Date: Fri, 29 Nov 2024 03:32:28 -0800 Subject: [PATCH 02/21] Versioning 3 functional tests --- common/dynamicconfig/constants.go | 10 - common/testing/taskpoller/taskpoller.go | 2 +- common/testing/testvars/test_vars.go | 13 + common/worker_versioning/worker_versioning.go | 6 +- .../historyservice/v1/request_response.proto | 12 +- .../api/recordworkflowtaskstarted/api.go | 4 - service/matching/config.go | 2 +- service/matching/matching_engine.go | 18 +- .../matching/physical_task_queue_manager.go | 74 ++- .../matching/task_queue_partition_manager.go | 99 ++-- service/matching/user_data_manager.go | 18 +- service/worker/deployment/fx.go | 2 +- tests/deployment_test.go | 4 +- tests/testcore/functional_test_base.go | 2 +- tests/versioning_3_test.go | 555 +++++++++++++++++- tests/versioning_test.go | 1 + 16 files changed, 694 insertions(+), 128 deletions(-) diff --git a/common/dynamicconfig/constants.go b/common/dynamicconfig/constants.go index 86665ecd438..4ae8d8a0e17 100644 --- a/common/dynamicconfig/constants.go +++ b/common/dynamicconfig/constants.go @@ -1220,11 +1220,6 @@ these log lines can be noisy, we want to be able to turn on and sample selective false, `MatchingDropNonRetryableTasks states if we should drop matching tasks with Internal/Dataloss errors`, ) - MatchingEnableDeployments = NewNamespaceBoolSetting( - "matching.enableDeployment", - false, - `MatchingEnableDeployments enables deployment-related RPCs in matching`, - ) MatchingMaxTaskQueuesInDeployment = NewNamespaceIntSetting( "matching.maxTaskQueuesInDeployment", 1000, @@ -2560,11 +2555,6 @@ If the service configures with archival feature enabled, update worker.historySc `How long to sleep within a local activity before pushing to workflow level sleep (don't make this close to or more than the workflow task timeout)`, ) - WorkerEnableDeployment = NewNamespaceBoolSetting( - "worker.enableDeployment", - false, - `WorkerEnableDeploymentGroup controls whether to start the worker for deployment and deployment-name workflows`, - ) WorkerDeleteNamespaceActivityLimits = NewGlobalTypedSetting( "worker.deleteNamespaceActivityLimitsConfig", sdkworker.Options{}, diff --git a/common/testing/taskpoller/taskpoller.go b/common/testing/taskpoller/taskpoller.go index 8b77a9370c8..7a8e579cf41 100644 --- a/common/testing/taskpoller/taskpoller.go +++ b/common/testing/taskpoller/taskpoller.go @@ -228,7 +228,7 @@ func (p *workflowTaskPoller) pollTask( if err != nil { return nil, err } - if resp == nil { + if resp == nil || resp.TaskToken == nil { return nil, NoWorkflowTaskAvailable } diff --git a/common/testing/testvars/test_vars.go b/common/testing/testvars/test_vars.go index 56e0d77c847..1e4d6728a2d 100644 --- a/common/testing/testvars/test_vars.go +++ b/common/testing/testvars/test_vars.go @@ -31,6 +31,7 @@ import ( "github.com/pborman/uuid" commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" namespacepb "go.temporal.io/api/namespace/v1" taskqueuepb "go.temporal.io/api/taskqueue/v1" @@ -157,11 +158,23 @@ func (tv *TestVars) BuildId(key ...string) string { return tv.getOrCreate("build_id", key).(string) } +func (tv *TestVars) WithBuildId(buildId string, key ...string) *TestVars { + return tv.cloneSet("build_id", key, buildId) +} + func (tv *TestVars) DeploymentSeries(key ...string) string { //revive:disable-next-line:unchecked-type-assertion return tv.getOrCreate("deployment_series", key).(string) } +func (tv *TestVars) Deployment(key ...string) *deployment.Deployment { + //revive:disable-next-line:unchecked-type-assertion + return &deployment.Deployment{ + SeriesName: tv.DeploymentSeries(key...), + BuildId: tv.BuildId(key...), + } +} + func (tv *TestVars) WithWorkflowID(workflowID string, key ...string) *TestVars { return tv.cloneSet("workflow_id", key, workflowID) } diff --git a/common/worker_versioning/worker_versioning.go b/common/worker_versioning/worker_versioning.go index 55767b23354..e30ed8e773f 100644 --- a/common/worker_versioning/worker_versioning.go +++ b/common/worker_versioning/worker_versioning.go @@ -218,10 +218,14 @@ func MakeBuildIdDirective(buildId string) *taskqueuespb.TaskVersionDirective { } func StampFromCapabilities(cap *commonpb.WorkerVersionCapabilities) *commonpb.WorkerVersionStamp { + if cap.GetUseVersioning() && cap.GetDeploymentSeriesName() != "" { + // Versioning 3, do not return stamp. + return nil + } // TODO: remove `cap.BuildId != ""` condition after old versioning cleanup. this condition is used to differentiate // between old and new versioning in Record*TaskStart calls. [cleanup-old-wv] // we don't want to add stamp for task started events in old versioning - if cap != nil && cap.BuildId != "" { + if cap.GetBuildId() != "" { return &commonpb.WorkerVersionStamp{UseVersioning: cap.UseVersioning, BuildId: cap.BuildId} } return nil diff --git a/proto/internal/temporal/server/api/historyservice/v1/request_response.proto b/proto/internal/temporal/server/api/historyservice/v1/request_response.proto index cdd31b48202..0da3bedbcd5 100644 --- a/proto/internal/temporal/server/api/historyservice/v1/request_response.proto +++ b/proto/internal/temporal/server/api/historyservice/v1/request_response.proto @@ -243,8 +243,7 @@ message RecordWorkflowTaskStartedRequest { temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest poll_request = 6; temporal.server.api.clock.v1.VectorClock clock = 7; temporal.server.api.taskqueue.v1.BuildIdRedirectInfo build_id_redirect_info = 8; - // Presence of this value means matching has redirected the task to a deployment other than - // the deployment that History passed when scheduling the task. + // The deployment passed by History when the task was scheduled. temporal.api.deployment.v1.Deployment scheduled_deployment = 9; } @@ -286,13 +285,12 @@ message RecordActivityTaskStartedRequest { // Stamp represents the internal “version” of the activity options and can/will be changed with Activity API. int32 stamp = 9; - // Presence of this value means matching has redirected the task to a deployment other than - // the deployment that History passed when scheduling the task. + // The deployment passed by History when the task was scheduled. temporal.api.deployment.v1.Deployment scheduled_deployment = 10; - // Whether the directive deployment contains the activity's task queue. Used by History to + // Whether the scheduled deployment contains the activity's task queue. Used by History to // determine if the activity redirect should affect the workflow. - // Only set if `directive_deployment` is set (i.e. the task is redirected). - bool directive_deployment_contains_task_queue = 11; + // Only set if `scheduled_deployment` is set (i.e. the task is redirected). + bool scheduled_deployment_contains_task_queue = 11; } message RecordActivityTaskStartedResponse { diff --git a/service/history/api/recordworkflowtaskstarted/api.go b/service/history/api/recordworkflowtaskstarted/api.go index d336cdfae2a..0c73d22cb25 100644 --- a/service/history/api/recordworkflowtaskstarted/api.go +++ b/service/history/api/recordworkflowtaskstarted/api.go @@ -171,10 +171,6 @@ func Invoke( pollerDeployment := worker_versioning.DeploymentFromCapabilities(req.PollRequest.WorkerVersionCapabilities) // Effective deployment of the workflow when History scheduled the WFT. scheduledDeployment := req.GetScheduledDeployment() - if scheduledDeployment == nil { - // Matching does not send the directive deployment when it's the same as poller's. - scheduledDeployment = pollerDeployment - } if !scheduledDeployment.Equal(wfDeployment) { // This must be an AT scheduled before the workflow transitions to the current // deployment. Matching can drop it. diff --git a/service/matching/config.go b/service/matching/config.go index 8ac73fd9146..fca1f13e5e3 100644 --- a/service/matching/config.go +++ b/service/matching/config.go @@ -214,7 +214,7 @@ func NewConfig( TestDisableSyncMatch: dynamicconfig.TestMatchingDisableSyncMatch.Get(dc), LoadUserData: dynamicconfig.MatchingLoadUserData.Get(dc), HistoryMaxPageSize: dynamicconfig.MatchingHistoryMaxPageSize.Get(dc), - EnableDeployments: dynamicconfig.MatchingEnableDeployments.Get(dc), + EnableDeployments: dynamicconfig.FrontendEnableDeployments.Get(dc), MaxTaskQueuesInDeployment: dynamicconfig.MatchingMaxTaskQueuesInDeployment.Get(dc), RPS: dynamicconfig.MatchingRPS.Get(dc), OperatorRPSRatio: dynamicconfig.OperatorRPSRatio.Get(dc), diff --git a/service/matching/matching_engine.go b/service/matching/matching_engine.go index de750e21fd2..affa38585e8 100644 --- a/service/matching/matching_engine.go +++ b/service/matching/matching_engine.go @@ -2406,14 +2406,7 @@ func (e *matchingEngineImpl) recordWorkflowTaskStarted( RequestId: uuid.New(), PollRequest: pollReq, BuildIdRedirectInfo: task.redirectInfo, - } - - scheduledDeployment := task.event.Data.VersionDirective.GetDeployment() - dispatchDeployment := worker_versioning.DeploymentFromCapabilities(pollReq.GetWorkerVersionCapabilities()) - if !scheduledDeployment.Equal(dispatchDeployment) { - // Redirect has happened, set the directive deployment in the request so History can - // validate the task is not stale. - recordStartedRequest.ScheduledDeployment = scheduledDeployment + ScheduledDeployment: task.event.Data.VersionDirective.GetDeployment(), } return e.historyClient.RecordWorkflowTaskStarted(ctx, recordStartedRequest) @@ -2436,14 +2429,7 @@ func (e *matchingEngineImpl) recordActivityTaskStarted( PollRequest: pollReq, BuildIdRedirectInfo: task.redirectInfo, Stamp: task.event.Data.GetStamp(), - } - - scheduledDeployment := task.event.Data.VersionDirective.GetDeployment() - dispatchDeployment := worker_versioning.DeploymentFromCapabilities(pollReq.GetWorkerVersionCapabilities()) - if !scheduledDeployment.Equal(dispatchDeployment) { - // Redirect has happened, set the directive deployment in the request so History can - // validate the task is not stale. - recordStartedRequest.ScheduledDeployment = scheduledDeployment + ScheduledDeployment: task.event.Data.VersionDirective.GetDeployment(), } return e.historyClient.RecordActivityTaskStarted(ctx, recordStartedRequest) diff --git a/service/matching/physical_task_queue_manager.go b/service/matching/physical_task_queue_manager.go index 24d40679848..8a588baf92a 100644 --- a/service/matching/physical_task_queue_manager.go +++ b/service/matching/physical_task_queue_manager.go @@ -38,6 +38,7 @@ import ( "go.temporal.io/api/serviceerror" taskqueuepb "go.temporal.io/api/taskqueue/v1" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/server/api/enums/v1" "go.temporal.io/server/api/matchingservice/v1" persistencespb "go.temporal.io/server/api/persistence/v1" taskqueuespb "go.temporal.io/server/api/taskqueue/v1" @@ -319,6 +320,15 @@ func (c *physicalTaskQueueManagerImpl) WaitUntilInitialized(ctx context.Context) func (c *physicalTaskQueueManagerImpl) SpoolTask(taskInfo *persistencespb.TaskInfo) error { c.liveness.markAlive() + if c.queue.version.Deployment() != nil { + fmt.Printf( + "\n %s shahab> spooled task %s %s %s\n\n", + time.Now(), + c.queue.partition.RpcName(), + c.queue.partition.TaskType().String(), + c.queue.version.Deployment(), + ) + } return c.backlogMgr.SpoolTask(taskInfo) } @@ -330,6 +340,15 @@ func (c *physicalTaskQueueManagerImpl) PollTask( ctx context.Context, pollMetadata *pollMetadata, ) (*internalTask, error) { + if c.queue.version.Deployment() != nil { + fmt.Printf( + "\n %s shahab> polling task %s %s %s\n\n", + time.Now(), + c.queue.partition.RpcName(), + c.queue.partition.TaskType().String(), + c.queue.version.Deployment(), + ) + } c.liveness.markAlive() c.currentPolls.Add(1) @@ -381,6 +400,17 @@ func (c *physicalTaskQueueManagerImpl) PollTask( (!task.isStarted() || !task.started.hasEmptyResponse()) { // Need to filter out the empty "started" ones c.tasksDispatchedInIntervals.incrementTaskCount() } + + if c.queue.version.Deployment() != nil { + fmt.Printf( + "\n %s shahab> polled task %s %s %s\n\n", + time.Now(), + c.queue.partition.RpcName(), + c.queue.partition.TaskType().String(), + c.queue.version.Deployment(), + ) + } + return task, nil } } @@ -535,7 +565,17 @@ func (c *physicalTaskQueueManagerImpl) TrySyncMatch(ctx context.Context, task *i childCtx, cancel := newChildContext(ctx, c.config.SyncMatchWaitDuration(), time.Second) defer cancel() - return c.matcher.Offer(childCtx, task) + a, b := c.matcher.Offer(childCtx, task) + if task.source == enums.TASK_SOURCE_HISTORY && a && c.queue.version.Deployment() != nil { + fmt.Printf( + "\n %s shahab> sync-matched task %s %s %s\n\n", + time.Now(), + c.queue.partition.RpcName(), + c.queue.partition.TaskType().String(), + c.queue.version.Deployment(), + ) + } + return a, b } func (c *physicalTaskQueueManagerImpl) ensureRegisteredInDeployment( @@ -595,22 +635,22 @@ func (c *physicalTaskQueueManagerImpl) ensureRegisteredInDeployment( // the deployment workflow will register itself in this task queue's user data. // wait for it to propagate here. - for { - userData, userDataChanged, err := c.partitionMgr.GetUserDataManager().GetUserData() - if err != nil { - return err - } - deploymentData := userData.GetData().GetPerType()[int32(c.queue.TaskType())].GetDeploymentData() - if findDeployment(deploymentData, workerDeployment) >= 0 { - break - } - select { - case <-userDataChanged: - case <-ctx.Done(): - c.logger.Error("timed out waiting for deployment to appear in user data") - return ctx.Err() - } - } + //for { + // userData, userDataChanged, err := c.partitionMgr.GetUserDataManager().GetUserData() + // if err != nil { + // return err + // } + // deploymentData := userData.GetData().GetPerType()[int32(c.queue.TaskType())].GetDeploymentData() + // if findDeployment(deploymentData, workerDeployment) >= 0 { + // break + // } + // select { + // case <-userDataChanged: + // case <-ctx.Done(): + // c.logger.Error("timed out waiting for deployment to appear in user data") + // return ctx.Err() + // } + //} c.deploymentRegistered = true return nil diff --git a/service/matching/task_queue_partition_manager.go b/service/matching/task_queue_partition_manager.go index 9d678e0e4b2..20dd92b74ad 100644 --- a/service/matching/task_queue_partition_manager.go +++ b/service/matching/task_queue_partition_manager.go @@ -781,62 +781,61 @@ func (pm *taskQueuePartitionManagerImpl) getPhysicalQueuesForAdd( forwardInfo *taskqueuespb.TaskForwardInfo, runId string, ) (pinnedQueue physicalTaskQueueManager, syncMatchQueue physicalTaskQueueManager, userDataChanged <-chan struct{}, err error) { - if deployment := directive.GetDeployment(); deployment != nil { - wfBehavior := directive.GetBehavior() - - switch wfBehavior { - case enumspb.VERSIONING_BEHAVIOR_PINNED: - if pm.partition.Kind() == enumspb.TASK_QUEUE_KIND_STICKY { - // TODO (shahab): we can verify the passed deployment matches the last poller's deployment - return pm.defaultQueue, pm.defaultQueue, userDataChanged, nil - } + wfBehavior := directive.GetBehavior() + deployment := directive.GetDeployment() - err = worker_versioning.ValidateDeployment(deployment) - if err != nil { - return nil, nil, nil, err - } - pinnedQueue, err = pm.getVersionedQueue(ctx, "", "", deployment, true) - if err != nil { - return nil, nil, nil, err - } - if forwardInfo == nil { - // Task is not forwarded, so it can be spooled if sync match fails. - // Spool queue and sync match queue is the same for pinned workflows. - return pinnedQueue, pinnedQueue, nil, nil - } else { - // Forwarded from child partition - only do sync match. - return nil, pinnedQueue, nil, nil - } - case enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE: - perTypeUserData, perTypeUserDataChanged, err := pm.getPerTypeUserData() - if err != nil { - return nil, nil, nil, err - } + if wfBehavior == enumspb.VERSIONING_BEHAVIOR_PINNED { + if pm.partition.Kind() == enumspb.TASK_QUEUE_KIND_STICKY { + // TODO (shahab): we can verify the passed deployment matches the last poller's deployment + return pm.defaultQueue, pm.defaultQueue, userDataChanged, nil + } - currentDeployment := findCurrentDeployment(perTypeUserData.GetDeploymentData()) + err = worker_versioning.ValidateDeployment(deployment) + if err != nil { + return nil, nil, nil, err + } + pinnedQueue, err = pm.getVersionedQueue(ctx, "", "", deployment, true) + if err != nil { + return nil, nil, nil, err + } + if forwardInfo == nil { + // Task is not forwarded, so it can be spooled if sync match fails. + // Spool queue and sync match queue is the same for pinned workflows. + return pinnedQueue, pinnedQueue, nil, nil + } else { + // Forwarded from child partition - only do sync match. + return nil, pinnedQueue, nil, nil + } + } - if pm.partition.Kind() == enumspb.TASK_QUEUE_KIND_STICKY { - if !deployment.Equal(currentDeployment) { - // Current deployment has changed, so the workflow should move to a normal queue to - // get redirected to the new deployment. - return nil, nil, nil, serviceerrors.NewStickyWorkerUnavailable() - } + perTypeUserData, perTypeUserDataChanged, err := pm.getPerTypeUserData() + if err != nil { + return nil, nil, nil, err + } - // TODO (shahab): we can verify the passed deployment matches the last poller's deployment - return pm.defaultQueue, pm.defaultQueue, perTypeUserDataChanged, nil + currentDeployment := findCurrentDeployment(perTypeUserData.GetDeploymentData()) + if currentDeployment != nil && + // Make sure the wf is not v1-2 versioned + directive.GetAssignedBuildId() == "" { + if pm.partition.Kind() == enumspb.TASK_QUEUE_KIND_STICKY { + if !deployment.Equal(currentDeployment) { + // Current deployment has changed, so the workflow should move to a normal queue to + // get redirected to the new deployment. + return nil, nil, nil, serviceerrors.NewStickyWorkerUnavailable() } - currentDeploymentQueue, err := pm.getVersionedQueue(ctx, "", "", currentDeployment, true) - if forwardInfo == nil { - // Task is not forwarded, so it can be spooled if sync match fails. - // Unpinned tasks are spooled in default queue - return pm.defaultQueue, currentDeploymentQueue, perTypeUserDataChanged, err - } else { - // Forwarded from child partition - only do sync match. - return nil, currentDeploymentQueue, perTypeUserDataChanged, err - } - case enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED: - return nil, nil, nil, serviceerror.NewInvalidArgument("versioning behavior must be set") + // TODO (shahab): we can verify the passed deployment matches the last poller's deployment + return pm.defaultQueue, pm.defaultQueue, perTypeUserDataChanged, nil + } + + currentDeploymentQueue, err := pm.getVersionedQueue(ctx, "", "", currentDeployment, true) + if forwardInfo == nil { + // Task is not forwarded, so it can be spooled if sync match fails. + // Unpinned tasks are spooled in default queue + return pm.defaultQueue, currentDeploymentQueue, perTypeUserDataChanged, err + } else { + // Forwarded from child partition - only do sync match. + return nil, currentDeploymentQueue, perTypeUserDataChanged, err } } diff --git a/service/matching/user_data_manager.go b/service/matching/user_data_manager.go index ccbb1615b4c..864cdfc8d0b 100644 --- a/service/matching/user_data_manager.go +++ b/service/matching/user_data_manager.go @@ -310,6 +310,7 @@ func (m *userDataManagerImpl) fetchUserData(ctx context.Context) error { return nil } + fastResponseCounter := 0 minWaitTime := m.config.GetUserDataMinWaitTime for ctx.Err() == nil { @@ -322,11 +323,20 @@ func (m *userDataManagerImpl) fetchUserData(ctx context.Context) error { // spinning. So enforce a minimum wait time that increases as long as we keep getting // very fast replies. if elapsed < m.config.GetUserDataMinWaitTime { - util.InterruptibleSleep(ctx, minWaitTime-elapsed) - // Don't let this get near our call timeout, otherwise we can't tell the difference - // between a fast reply and a timeout. - minWaitTime = min(minWaitTime*2, m.config.GetUserDataLongPollTimeout()/2) + if fastResponseCounter >= 3 { + // 3 or more consecutive fast responses, let's throttle! + util.InterruptibleSleep(ctx, minWaitTime-elapsed) + // Don't let this get near our call timeout, otherwise we can't tell the difference + // between a fast reply and a timeout. + minWaitTime = min(minWaitTime*2, m.config.GetUserDataLongPollTimeout()/2) + } else { + // Not yet 3 consecutive fast responses. A few rapid refreshes for versioned queues + // is expected when the first poller arrives. We do not want to slow down the queue + // for that. + fastResponseCounter++ + } } else { + fastResponseCounter = 0 minWaitTime = m.config.GetUserDataMinWaitTime } } diff --git a/service/worker/deployment/fx.go b/service/worker/deployment/fx.go index f93cde5e0a7..ada8caf4c77 100644 --- a/service/worker/deployment/fx.go +++ b/service/worker/deployment/fx.go @@ -109,7 +109,7 @@ func NewResult( return fxResult{ Component: &workerComponent{ activityDeps: params, - enabledForNs: dynamicconfig.WorkerEnableDeployment.Get(dc), + enabledForNs: dynamicconfig.FrontendEnableDeployments.Get(dc), }, } } diff --git a/tests/deployment_test.go b/tests/deployment_test.go index d908a84be61..2d246b4230c 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -87,8 +87,6 @@ func (s *DeploymentSuite) SetupSuite() { dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningRuleAPIs.Key(): true, dynamicconfig.FrontendEnableExecuteMultiOperation.Key(): true, - dynamicconfig.MatchingEnableDeployments.Key(): true, - dynamicconfig.WorkerEnableDeployment.Key(): true, // Reachability dynamicconfig.ReachabilityCacheOpenWFsTTL.Key(): testReachabilityCacheOpenWFsTTL, @@ -1122,5 +1120,5 @@ func (s *DeploymentSuite) TestSignalWithStartWorkflowExecution_WithUnpinnedOverr // do not grow larger that DB column limit (currently as low as 272 chars). func (s *DeploymentSuite) Name() string { fullName := s.T().Name() - return fullName[len(fullName)-20:] + return fullName[len(fullName)-30:] } diff --git a/tests/testcore/functional_test_base.go b/tests/testcore/functional_test_base.go index 6d57aa59847..1d70b7385b6 100644 --- a/tests/testcore/functional_test_base.go +++ b/tests/testcore/functional_test_base.go @@ -184,7 +184,7 @@ func (s *FunctionalTestBase) SetupSuite(defaultClusterConfigFile string, options s.operatorClient = s.testCluster.OperatorClient() s.httpAPIAddress = cluster.Host().FrontendHTTPAddress() - s.namespace = RandomizeStr("functional-test-namespace") + s.namespace = "functional-test-namespace" s.Require().NoError(s.registerNamespaceWithDefaults(s.namespace)) s.foreignNamespace = RandomizeStr("functional-foreign-test-namespace") diff --git a/tests/versioning_3_test.go b/tests/versioning_3_test.go index 1c6620767e3..35c2279024f 100644 --- a/tests/versioning_3_test.go +++ b/tests/versioning_3_test.go @@ -25,27 +25,423 @@ package tests import ( + "context" + "fmt" + "testing" "time" + "github.com/dgryski/go-farm" "github.com/pborman/uuid" + "github.com/stretchr/testify/suite" + commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" + deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/api/workflow/v1" "go.temporal.io/api/workflowservice/v1" + deploymentspb "go.temporal.io/server/api/deployment/v1" + "go.temporal.io/server/api/historyservice/v1" + "go.temporal.io/server/api/matchingservice/v1" + "go.temporal.io/server/common" + "go.temporal.io/server/common/dynamicconfig" + "go.temporal.io/server/common/payloads" + "go.temporal.io/server/common/primitives/timestamp" + "go.temporal.io/server/common/testing/taskpoller" + "go.temporal.io/server/common/testing/testvars" + "go.temporal.io/server/common/tqid" "go.temporal.io/server/tests/testcore" "google.golang.org/protobuf/types/known/durationpb" ) -func (s *VersioningIntegSuite) TestPinnedWorkflow() { - // TODO (shahab) implement this with TaskPoller etc. - s.startWorkflow() +var ( + unspecified = enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED + pinned = enumspb.VERSIONING_BEHAVIOR_PINNED + unpinned = enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE + unpinnedOverride = &workflow.VersioningOverride{Behavior: unpinned} + minPollTime = common.MinLongPollTimeout + time.Millisecond*200 +) + +func pinnedOverride(d *deploymentpb.Deployment) *workflow.VersioningOverride { + return &workflow.VersioningOverride{Behavior: pinned, Deployment: d} +} + +type Versioning3Suite struct { + // override suite.Suite.Assertions with require.Assertions; this means that s.NotNil(nil) will stop the test, + // not merely log an error + testcore.FunctionalTestBase +} + +func TestVersioning3FunctionalSuite(t *testing.T) { + t.Parallel() + suite.Run(t, new(Versioning3Suite)) +} + +func (s *Versioning3Suite) SetupSuite() { + dynamicConfigOverrides := map[dynamicconfig.Key]any{ + dynamicconfig.FrontendEnableDeployments.Key(): true, + dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, + dynamicconfig.MatchingForwarderMaxChildrenPerNode.Key(): partitionTreeDegree, + + // Make sure we don't hit the rate limiter in tests + dynamicconfig.FrontendGlobalNamespaceNamespaceReplicationInducingAPIsRPS.Key(): 1000, + dynamicconfig.FrontendMaxNamespaceNamespaceReplicationInducingAPIsBurstRatioPerInstance.Key(): 1, + dynamicconfig.FrontendNamespaceReplicationInducingAPIsRPS.Key(): 1000, + + // this is overridden for tests using RunTestWithMatchingBehavior + dynamicconfig.MatchingNumTaskqueueReadPartitions.Key(): 4, + dynamicconfig.MatchingNumTaskqueueWritePartitions.Key(): 4, + } + s.SetDynamicConfigOverrides(dynamicConfigOverrides) + s.FunctionalTestBase.SetupSuite("testdata/es_cluster.yaml") +} + +func (s *Versioning3Suite) TearDownSuite() { + s.FunctionalTestBase.TearDownSuite() +} + +func (s *Versioning3Suite) SetupTest() { + s.FunctionalTestBase.SetupTest() +} + +func (s *Versioning3Suite) TestPinnedTask_NoProperPoller() { + s.RunTestWithMatchingBehavior( + func() { + tv := testvars.New(s) + + s.startWorkflow(tv, pinnedOverride(tv.Deployment())) + + go func() { + s.idlePollWorkflow(tv.WithBuildId("other"), true, minPollTime, "other deployment should not receive pinned task") + }() + + s.idlePollWorkflow(tv, false, minPollTime, "unversioned worker should not receive pinned task") + }) +} + +func (s *Versioning3Suite) TestUnpinnedTask_NonCurrentDeployment() { + s.RunTestWithMatchingBehavior( + func() { + tv := testvars.New(s) + s.startWorkflow(tv, nil) + s.idlePollWorkflow(tv, true, minPollTime, "non-current versioned poller should not receive unpinned task") + }) +} + +func (s *Versioning3Suite) TestUnpinnedTask_OldDeployment() { + s.RunTestWithMatchingBehavior( + func() { + tv := testvars.New(s) + // previous current deployment + s.updateTaskQueueDeploymentData(tv.WithBuildId("older"), time.Minute) + // current deployment + s.updateTaskQueueDeploymentData(tv, 0) + + s.startWorkflow(tv, nil) + + s.idlePollWorkflow( + tv.WithBuildId("older"), + true, + minPollTime, + "old deployment should not receive unpinned task", + ) + }, + ) +} + +func (s *Versioning3Suite) TestUnpinnedWorkflow_Sticky() { + s.RunTestWithMatchingBehavior( + func() { + s.testUnpinnedWorkflow(true) + }, + ) +} + +func (s *Versioning3Suite) TestUnpinnedWorkflow_NoSticky() { + s.RunTestWithMatchingBehavior( + func() { + s.testUnpinnedWorkflow(false) + }, + ) } -func (s *VersioningIntegSuite) startWorkflow() *commonpb.WorkflowExecution { - id := s.randomizeName("my-wf") +func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { + tv := testvars.New(s) + + if sticky { + s.warmUpSticky(tv) + } + + // current deployment + s.updateTaskQueueDeploymentData(tv, 0) + + we := s.startWorkflow(tv, nil) + + _, err := s.pollWftAndHandle(tv, false, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + s.verifyWorkflowVersioning(we, unspecified, nil, nil) + return respondWftWithActivity(tv, sticky, unpinned, "5"), nil + }) + s.NoError(err) + s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), nil) + if sticky { + s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) + } + + _, err = s.pollActivityAndHandle(tv, + func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { + s.NotNil(task) + return respondActivity(), nil + }) + s.NoError(err) + s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), nil) + + _, err = s.pollWftAndHandle(tv, sticky, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + return respondCompleteWorkflow(tv, unpinned), nil + }) + s.NoError(err) + s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), nil) +} + +func (s *Versioning3Suite) TestWorkflowWithPinnedOverride_Sticky() { + s.RunTestWithMatchingBehavior( + func() { + s.testWorkflowWithPinnedOverride(true) + }, + ) +} + +func (s *Versioning3Suite) TestWorkflowWithPinnedOverride_NoSticky() { + s.RunTestWithMatchingBehavior( + func() { + s.testWorkflowWithPinnedOverride(false) + }, + ) +} + +func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { + tv := testvars.New(s) + + if sticky { + s.warmUpSticky(tv) + } + + override := pinnedOverride(tv.Deployment()) + we := s.startWorkflow(tv, override) + + _, err := s.pollWftAndHandle(tv, false, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + return respondWftWithActivity(tv, sticky, unpinned, "5"), nil + }) + s.NoError(err) + s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), override) + if sticky { + s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) + } + + _, err = s.pollActivityAndHandle(tv, + func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { + s.NotNil(task) + return respondActivity(), nil + }) + s.NoError(err) + s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), override) + + _, err = s.pollWftAndHandle(tv, sticky, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + return respondCompleteWorkflow(tv, unpinned), nil + }) + s.NoError(err) + s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), override) +} + +func (s *Versioning3Suite) updateTaskQueueDeploymentData( + tv *testvars.TestVars, + timeSinceCurrent time.Duration, +) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + defer cancel() + + lastBecameCurrent := time.Now().Add(-timeSinceCurrent) + types := []enumspb.TaskQueueType{enumspb.TASK_QUEUE_TYPE_WORKFLOW, enumspb.TASK_QUEUE_TYPE_ACTIVITY} + for _, t := range types { + _, err := s.GetTestCluster().MatchingClient().SyncDeploymentUserData( + ctx, &matchingservice.SyncDeploymentUserDataRequest{ + NamespaceId: s.GetNamespaceID(s.Namespace()), + TaskQueue: tv.TaskQueue().GetName(), + TaskQueueType: t, + Deployment: tv.Deployment(), + Data: &deploymentspb.TaskQueueData{ + FirstPollerTime: timestamp.TimePtr(lastBecameCurrent), + LastBecameCurrentTime: timestamp.TimePtr(lastBecameCurrent), + }, + }, + ) + s.NoError(err) + } + s.waitForDeploymentDataPropagation(ctx, tv) +} + +func (s *Versioning3Suite) waitForDeploymentDataPropagation( + ctx context.Context, + tv *testvars.TestVars, +) { + v := s.GetTestCluster().Host().DcClient().GetValue(dynamicconfig.MatchingNumTaskqueueReadPartitions.Key()) + s.NotEmpty(v, "versioning tests require setting explicit number of partitions") + count, ok := v[0].Value.(int) + s.True(ok, "partition count is not an int") + partitionCount := count + + type partAndType struct { + part int + tp enumspb.TaskQueueType + } + remaining := make(map[partAndType]struct{}) + for i := 0; i < partitionCount; i++ { + remaining[partAndType{i, enumspb.TASK_QUEUE_TYPE_ACTIVITY}] = struct{}{} + remaining[partAndType{i, enumspb.TASK_QUEUE_TYPE_WORKFLOW}] = struct{}{} + } + nsId := s.GetNamespaceID(s.Namespace()) + f, err := tqid.NewTaskQueueFamily(nsId, tv.TaskQueue().GetName()) + deployment := tv.Deployment() + s.Eventually(func() bool { + for pt := range remaining { + s.NoError(err) + partition := f.TaskQueue(pt.tp).NormalPartition(pt.part) + // Use lower-level GetTaskQueueUserData instead of GetWorkerBuildIdCompatibility + // here so that we can target activity queues. + res, err := s.GetTestCluster().MatchingClient().GetTaskQueueUserData( + ctx, + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: nsId, + TaskQueue: partition.RpcName(), + TaskQueueType: partition.TaskType(), + }) + s.NoError(err) + perTypes := res.GetUserData().GetData().GetPerType() + if perTypes != nil { + deps := perTypes[int32(pt.tp)].GetDeploymentData().GetDeployments() + for _, d := range deps { + if d.GetDeployment().Equal(deployment) { + delete(remaining, pt) + } + } + } + } + return len(remaining) == 0 + }, 10*time.Second, 100*time.Millisecond) +} + +func (s *Versioning3Suite) verifyWorkflowVersioning( + we *commonpb.WorkflowExecution, + behavior enumspb.VersioningBehavior, + deployment *deploymentpb.Deployment, + override *workflow.VersioningOverride, +) { + dwf, err := s.FrontendClient().DescribeWorkflowExecution( + context.Background(), &workflowservice.DescribeWorkflowExecutionRequest{ + Namespace: s.Namespace(), + Execution: we, + }, + ) + s.NoError(err) + + versioningInfo := dwf.WorkflowExecutionInfo.GetVersioningInfo() + s.Equal(behavior.String(), versioningInfo.GetBehavior().String()) + if !deployment.Equal(versioningInfo.GetDeployment()) { + s.Fail(fmt.Sprintf("deployment mismatch. expected: {%s}, actual: {%s}", + deployment, + versioningInfo.GetDeployment(), + )) + } + + s.Equal(override.GetBehavior().String(), versioningInfo.GetVersioningOverride().GetBehavior().String()) + if actualOverrideDeployment := versioningInfo.GetVersioningOverride().GetDeployment(); !override.GetDeployment().Equal(actualOverrideDeployment) { + s.Fail(fmt.Sprintf("deployment override mismatch. expected: {%s}, actual: {%s}", + override.GetDeployment(), + actualOverrideDeployment, + )) + } +} + +func respondActivity() *workflowservice.RespondActivityTaskCompletedRequest { + return &workflowservice.RespondActivityTaskCompletedRequest{} +} + +func respondWftWithActivity( + tv *testvars.TestVars, + sticky bool, + behavior enumspb.VersioningBehavior, + activityId string, +) *workflowservice.RespondWorkflowTaskCompletedRequest { + var stickyAttr *taskqueuepb.StickyExecutionAttributes + if sticky { + stickyAttr = &taskqueuepb.StickyExecutionAttributes{ + WorkerTaskQueue: tv.StickyTaskQueue(), + ScheduleToStartTimeout: durationpb.New(5 * time.Second), + } + } + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: []*commandpb.Command{ + { + CommandType: enumspb.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK, + Attributes: &commandpb.Command_ScheduleActivityTaskCommandAttributes{ + ScheduleActivityTaskCommandAttributes: &commandpb.ScheduleActivityTaskCommandAttributes{ + ActivityId: activityId, + ActivityType: &commonpb.ActivityType{Name: "act"}, + TaskQueue: tv.TaskQueue(), + Input: payloads.EncodeString("input"), + // TODO (shahab): tests with forced task forward take multiple seconds. Need to know why? + ScheduleToCloseTimeout: durationpb.New(10 * time.Second), + ScheduleToStartTimeout: durationpb.New(10 * time.Second), + StartToCloseTimeout: durationpb.New(1 * time.Second), + HeartbeatTimeout: durationpb.New(1 * time.Second), + RequestEagerExecution: false, + }, + }, + }, + }, + StickyAttributes: stickyAttr, + ReturnNewWorkflowTask: false, + ForceCreateNewWorkflowTask: false, + Deployment: tv.Deployment(), + VersioningBehavior: behavior, + } +} + +func respondCompleteWorkflow( + tv *testvars.TestVars, + behavior enumspb.VersioningBehavior, +) *workflowservice.RespondWorkflowTaskCompletedRequest { + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: []*commandpb.Command{ + { + CommandType: enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION, + Attributes: &commandpb.Command_CompleteWorkflowExecutionCommandAttributes{ + CompleteWorkflowExecutionCommandAttributes: &commandpb.CompleteWorkflowExecutionCommandAttributes{ + Result: payloads.EncodeString("done"), + }, + }, + }, + }, + ReturnNewWorkflowTask: false, + ForceCreateNewWorkflowTask: false, + Deployment: tv.Deployment(), + VersioningBehavior: behavior, + } +} + +func (s *Versioning3Suite) startWorkflow( + tv *testvars.TestVars, + override *workflow.VersioningOverride, +) *commonpb.WorkflowExecution { + id := tv.WorkflowID() wt := "MyWfType" - tqName := "my-tq" - tq := &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} identity := "worker1" request := &workflowservice.StartWorkflowExecutionRequest{ @@ -53,11 +449,12 @@ func (s *VersioningIntegSuite) startWorkflow() *commonpb.WorkflowExecution { Namespace: s.Namespace(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, - TaskQueue: tq, + TaskQueue: tv.TaskQueue(), Input: nil, WorkflowRunTimeout: durationpb.New(100 * time.Second), - WorkflowTaskTimeout: durationpb.New(1 * time.Second), + WorkflowTaskTimeout: durationpb.New(10 * time.Second), Identity: identity, + VersioningOverride: override, } we, err0 := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) @@ -68,7 +465,141 @@ func (s *VersioningIntegSuite) startWorkflow() *commonpb.WorkflowExecution { } } -// Adds the test name and a random string as postfix to the given name -func (s *VersioningIntegSuite) randomizeName(name string) string { - return testcore.RandomizeStr(name + "_" + s.T().Name()) +// Name is used by testvars. We use a shorten test name in variables so that physical task queue IDs +// do not grow larger that DB column limit (currently as low as 272 chars). +func (s *Versioning3Suite) Name() string { + fullName := s.T().Name() + if len(fullName) <= 300 { + return fullName + } + return fmt.Sprintf("%s-%08x", + fullName[len(fullName)-25:], + farm.Fingerprint32([]byte(fullName)), + ) +} + +func (s *Versioning3Suite) pollWftAndHandle( + tv *testvars.TestVars, + sticky bool, + handler func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error), +) (*workflowservice.RespondWorkflowTaskCompletedResponse, error) { + poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) + d := tv.Deployment() + tq := tv.TaskQueue() + if sticky { + tq = tv.StickyTaskQueue() + } + return poller.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{ + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: d.BuildId, + DeploymentSeriesName: d.SeriesName, + UseVersioning: true, + }, + TaskQueue: tq, + }, + ).HandleTask(tv, handler) +} + +func (s *Versioning3Suite) pollActivityAndHandle( + tv *testvars.TestVars, + handler func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error), +) (*workflowservice.RespondActivityTaskCompletedResponse, error) { + poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) + d := tv.Deployment() + return poller.PollActivityTask( + &workflowservice.PollActivityTaskQueueRequest{ + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: d.BuildId, + DeploymentSeriesName: d.SeriesName, + UseVersioning: true, + }, + }, + ).HandleTask(tv, handler, taskpoller.WithTimeout(time.Minute)) +} + +func (s *Versioning3Suite) idlePollWorkflow( + tv *testvars.TestVars, + versioned bool, + timeout time.Duration, + unexpectedTaskMessage string, +) { + poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) + d := tv.Deployment() + poller.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{ + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: d.BuildId, + DeploymentSeriesName: d.SeriesName, + UseVersioning: versioned, + }, + }, + ).HandleTask( + tv, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.Fail(unexpectedTaskMessage) + return nil, nil + }, + taskpoller.WithTimeout(timeout), + ) +} + +func (s *Versioning3Suite) idlePollActivity( + tv *testvars.TestVars, + versioned bool, + timeout time.Duration, + unexpectedTaskMessage string, +) { + poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) + d := tv.Deployment() + poller.PollActivityTask( + &workflowservice.PollActivityTaskQueueRequest{ + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: d.BuildId, + DeploymentSeriesName: d.SeriesName, + UseVersioning: versioned, + }, + }, + ).HandleTask( + tv, + func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { + s.Fail(unexpectedTaskMessage) + return nil, nil + }, + taskpoller.WithTimeout(timeout), + ) +} + +func (s *Versioning3Suite) verifyWorkflowStickyQueue( + we *commonpb.WorkflowExecution, + stickyQ *taskqueuepb.TaskQueue, +) { + ms, err := s.GetTestCluster().HistoryClient().GetMutableState( + context.Background(), &historyservice.GetMutableStateRequest{ + NamespaceId: s.GetNamespaceID(s.Namespace()), + Execution: we, + }, + ) + s.NoError(err) + s.Equal(stickyQ.GetName(), ms.StickyTaskQueue.GetName()) +} + +// Sticky queue needs to be created in server before tasks can schedule in it. Call to this method +// create the sticky queue by polling it. +func (s *Versioning3Suite) warmUpSticky( + tv *testvars.TestVars, +) { + poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) + poller.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{ + TaskQueue: tv.StickyTaskQueue(), + }, + ).HandleTask( + tv, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.Fail("sticky task is not expected") + return nil, nil + }, + taskpoller.WithTimeout(minPollTime), + ) } diff --git a/tests/versioning_test.go b/tests/versioning_test.go index fcead1ca710..13a54649393 100644 --- a/tests/versioning_test.go +++ b/tests/versioning_test.go @@ -86,6 +86,7 @@ func TestVersioningFunctionalSuite(t *testing.T) { func (s *VersioningIntegSuite) SetupSuite() { dynamicConfigOverrides := map[dynamicconfig.Key]any{ + dynamicconfig.FrontendEnableDeployments.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningDataAPIs.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningRuleAPIs.Key(): true, From 62a625c1fb0635da57dbcda8eb10d797b7377fd7 Mon Sep 17 00:00:00 2001 From: ShahabT Date: Fri, 29 Nov 2024 13:07:38 -0800 Subject: [PATCH 03/21] improved tests --- api/historyservice/v1/request_response.pb.go | 20 +- .../matching/physical_task_queue_manager.go | 32 +- tests/deployment_test.go | 55 ++++ tests/versioning_3_test.go | 293 +++++++++++------- 4 files changed, 258 insertions(+), 142 deletions(-) diff --git a/api/historyservice/v1/request_response.pb.go b/api/historyservice/v1/request_response.pb.go index 8396a9e7bab..5d32cf0672f 100644 --- a/api/historyservice/v1/request_response.pb.go +++ b/api/historyservice/v1/request_response.pb.go @@ -1132,8 +1132,7 @@ type RecordWorkflowTaskStartedRequest struct { PollRequest *v1.PollWorkflowTaskQueueRequest `protobuf:"bytes,6,opt,name=poll_request,json=pollRequest,proto3" json:"poll_request,omitempty"` Clock *v16.VectorClock `protobuf:"bytes,7,opt,name=clock,proto3" json:"clock,omitempty"` BuildIdRedirectInfo *v111.BuildIdRedirectInfo `protobuf:"bytes,8,opt,name=build_id_redirect_info,json=buildIdRedirectInfo,proto3" json:"build_id_redirect_info,omitempty"` - // Presence of this value means matching has redirected the task to a deployment other than - // the deployment that History passed when scheduling the task. + // The deployment passed by History when the task was scheduled. ScheduledDeployment *v112.Deployment `protobuf:"bytes,9,opt,name=scheduled_deployment,json=scheduledDeployment,proto3" json:"scheduled_deployment,omitempty"` } @@ -1423,13 +1422,12 @@ type RecordActivityTaskStartedRequest struct { BuildIdRedirectInfo *v111.BuildIdRedirectInfo `protobuf:"bytes,8,opt,name=build_id_redirect_info,json=buildIdRedirectInfo,proto3" json:"build_id_redirect_info,omitempty"` // Stamp represents the internal “version” of the activity options and can/will be changed with Activity API. Stamp int32 `protobuf:"varint,9,opt,name=stamp,proto3" json:"stamp,omitempty"` - // Presence of this value means matching has redirected the task to a deployment other than - // the deployment that History passed when scheduling the task. + // The deployment passed by History when the task was scheduled. ScheduledDeployment *v112.Deployment `protobuf:"bytes,10,opt,name=scheduled_deployment,json=scheduledDeployment,proto3" json:"scheduled_deployment,omitempty"` - // Whether the directive deployment contains the activity's task queue. Used by History to + // Whether the scheduled deployment contains the activity's task queue. Used by History to // determine if the activity redirect should affect the workflow. - // Only set if `directive_deployment` is set (i.e. the task is redirected). - DirectiveDeploymentContainsTaskQueue bool `protobuf:"varint,11,opt,name=directive_deployment_contains_task_queue,json=directiveDeploymentContainsTaskQueue,proto3" json:"directive_deployment_contains_task_queue,omitempty"` + // Only set if `scheduled_deployment` is set (i.e. the task is redirected). + ScheduledDeploymentContainsTaskQueue bool `protobuf:"varint,11,opt,name=scheduled_deployment_contains_task_queue,json=scheduledDeploymentContainsTaskQueue,proto3" json:"scheduled_deployment_contains_task_queue,omitempty"` } func (x *RecordActivityTaskStartedRequest) Reset() { @@ -1527,9 +1525,9 @@ func (x *RecordActivityTaskStartedRequest) GetScheduledDeployment() *v112.Deploy return nil } -func (x *RecordActivityTaskStartedRequest) GetDirectiveDeploymentContainsTaskQueue() bool { +func (x *RecordActivityTaskStartedRequest) GetScheduledDeploymentContainsTaskQueue() bool { if x != nil { - return x.DirectiveDeploymentContainsTaskQueue + return x.ScheduledDeploymentContainsTaskQueue } return false } @@ -10206,10 +10204,10 @@ var file_temporal_server_api_historyservice_v1_request_response_proto_rawDesc = 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x13, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, - 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x28, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x28, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x0b, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x24, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x44, 0x65, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x24, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x24, 0x92, 0xc4, 0x03, 0x20, 0x2a, 0x1e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, diff --git a/service/matching/physical_task_queue_manager.go b/service/matching/physical_task_queue_manager.go index 8a588baf92a..d134f336d75 100644 --- a/service/matching/physical_task_queue_manager.go +++ b/service/matching/physical_task_queue_manager.go @@ -635,22 +635,22 @@ func (c *physicalTaskQueueManagerImpl) ensureRegisteredInDeployment( // the deployment workflow will register itself in this task queue's user data. // wait for it to propagate here. - //for { - // userData, userDataChanged, err := c.partitionMgr.GetUserDataManager().GetUserData() - // if err != nil { - // return err - // } - // deploymentData := userData.GetData().GetPerType()[int32(c.queue.TaskType())].GetDeploymentData() - // if findDeployment(deploymentData, workerDeployment) >= 0 { - // break - // } - // select { - // case <-userDataChanged: - // case <-ctx.Done(): - // c.logger.Error("timed out waiting for deployment to appear in user data") - // return ctx.Err() - // } - //} + for { + userData, userDataChanged, err := c.partitionMgr.GetUserDataManager().GetUserData() + if err != nil { + return err + } + deploymentData := userData.GetData().GetPerType()[int32(c.queue.TaskType())].GetDeploymentData() + if findDeployment(deploymentData, workerDeployment) >= 0 { + break + } + select { + case <-userDataChanged: + case <-ctx.Done(): + c.logger.Error("timed out waiting for deployment to appear in user data") + return ctx.Err() + } + } c.deploymentRegistered = true return nil diff --git a/tests/deployment_test.go b/tests/deployment_test.go index 2d246b4230c..10823ff33c1 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -47,6 +47,7 @@ import ( "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/testing/testvars" + "go.temporal.io/server/common/tqid" deploymentwf "go.temporal.io/server/service/worker/deployment" "go.temporal.io/server/tests/testcore" "google.golang.org/protobuf/proto" @@ -191,6 +192,60 @@ func (s *DeploymentSuite) TestDescribeDeployment_RegisterTaskQueue() { }, time.Second*5, time.Millisecond*200) } +func (s *DeploymentSuite) TestDescribeDeployment_RegisterTaskQueue_ConcurrentPollers() { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + + // presence of internally used delimiters (:) or escape + // characters shouldn't break functionality + seriesName := testcore.RandomizeStr("my-series|:|:") + buildID := testcore.RandomizeStr("bgt:|") + + taskQueue := &taskqueuepb.TaskQueue{Name: "deployment-test", Kind: enumspb.TASK_QUEUE_KIND_NORMAL} + workerDeployment := &deploymentpb.Deployment{ + SeriesName: seriesName, + BuildId: buildID, + } + numberOfDeployments := 1 + + root, err := tqid.PartitionFromProto(taskQueue, s.Namespace(), enumspb.TASK_QUEUE_TYPE_WORKFLOW) + s.NoError(err) + // Making concurrent polls to 4 partitions, 3 polls to each + for p := 0; p < 4; p++ { + for i := 0; i < 3; i++ { + tq := &taskqueuepb.TaskQueue{Name: root.TaskQueue().NormalPartition(p).RpcName(), Kind: enumspb.TASK_QUEUE_KIND_NORMAL} + go func() { + s.pollFromDeployment(ctx, tq, workerDeployment) + }() + } + } + + // Querying the Deployment + s.EventuallyWithT(func(t *assert.CollectT) { + a := assert.New(t) + + resp, err := s.FrontendClient().DescribeDeployment(ctx, &workflowservice.DescribeDeploymentRequest{ + Namespace: s.Namespace(), + Deployment: workerDeployment, + }) + if !a.NoError(err) { + return + } + a.NotNil(resp.GetDeploymentInfo()) + a.NotNil(resp.GetDeploymentInfo().GetDeployment()) + + a.Equal(seriesName, resp.GetDeploymentInfo().GetDeployment().GetSeriesName()) + a.Equal(buildID, resp.GetDeploymentInfo().GetDeployment().GetBuildId()) + + if !a.Equal(numberOfDeployments, len(resp.GetDeploymentInfo().GetTaskQueueInfos())) { + return + } + a.Equal(taskQueue.Name, resp.GetDeploymentInfo().GetTaskQueueInfos()[0].Name) + a.Equal(false, resp.GetDeploymentInfo().GetIsCurrent()) + // todo (Shivam) - please add a check for current time + }, time.Second*5, time.Millisecond*1000) +} + func (s *DeploymentSuite) TestGetCurrentDeployment_NoCurrentDeployment() { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() diff --git a/tests/versioning_3_test.go b/tests/versioning_3_test.go index 35c2279024f..0b2ee3a218e 100644 --- a/tests/versioning_3_test.go +++ b/tests/versioning_3_test.go @@ -55,6 +55,8 @@ import ( ) var ( + tqTypeWf = enumspb.TASK_QUEUE_TYPE_WORKFLOW + tqTypeAct = enumspb.TASK_QUEUE_TYPE_ACTIVITY unspecified = enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED pinned = enumspb.VERSIONING_BEHAVIOR_PINNED unpinned = enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE @@ -109,12 +111,13 @@ func (s *Versioning3Suite) TestPinnedTask_NoProperPoller() { func() { tv := testvars.New(s) - s.startWorkflow(tv, pinnedOverride(tv.Deployment())) - + other := tv.WithBuildId("other") go func() { - s.idlePollWorkflow(tv.WithBuildId("other"), true, minPollTime, "other deployment should not receive pinned task") + s.idlePollWorkflow(other, true, minPollTime, "other deployment should not receive pinned task") }() + s.waitForDeploymentDataPropagation(other, tqTypeWf) + s.startWorkflow(tv, pinnedOverride(tv.Deployment())) s.idlePollWorkflow(tv, false, minPollTime, "unversioned worker should not receive pinned task") }) } @@ -123,8 +126,12 @@ func (s *Versioning3Suite) TestUnpinnedTask_NonCurrentDeployment() { s.RunTestWithMatchingBehavior( func() { tv := testvars.New(s) + go func() { + s.idlePollWorkflow(tv, true, minPollTime, "non-current versioned poller should not receive unpinned task") + }() + s.waitForDeploymentDataPropagation(tv, tqTypeWf) + s.startWorkflow(tv, nil) - s.idlePollWorkflow(tv, true, minPollTime, "non-current versioned poller should not receive unpinned task") }) } @@ -133,9 +140,9 @@ func (s *Versioning3Suite) TestUnpinnedTask_OldDeployment() { func() { tv := testvars.New(s) // previous current deployment - s.updateTaskQueueDeploymentData(tv.WithBuildId("older"), time.Minute) + s.updateTaskQueueDeploymentData(tv.WithBuildId("older"), time.Minute, tqTypeWf) // current deployment - s.updateTaskQueueDeploymentData(tv, 0) + s.updateTaskQueueDeploymentData(tv, 0, tqTypeWf) s.startWorkflow(tv, nil) @@ -167,43 +174,54 @@ func (s *Versioning3Suite) TestUnpinnedWorkflow_NoSticky() { func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { tv := testvars.New(s) + d := tv.Deployment() if sticky { s.warmUpSticky(tv) } - // current deployment - s.updateTaskQueueDeploymentData(tv, 0) - - we := s.startWorkflow(tv, nil) - - _, err := s.pollWftAndHandle(tv, false, + wftCompleted := make(chan interface{}) + s.pollWftAndHandle(tv, false, wftCompleted, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - s.verifyWorkflowVersioning(we, unspecified, nil, nil) + s.verifyWorkflowVersioning(tv, unspecified, nil, nil, transitionTo(d)) return respondWftWithActivity(tv, sticky, unpinned, "5"), nil }) - s.NoError(err) - s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), nil) - if sticky { - s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) - } + s.waitForDeploymentDataPropagation(tv, tqTypeWf) - _, err = s.pollActivityAndHandle(tv, + actCompleted := make(chan interface{}) + s.pollActivityAndHandle(tv, actCompleted, func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { s.NotNil(task) return respondActivity(), nil }) - s.NoError(err) - s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), nil) + s.waitForDeploymentDataPropagation(tv, tqTypeAct) + + s.updateTaskQueueDeploymentData(tv, 0, tqTypeWf, tqTypeAct) + + we := s.startWorkflow(tv, nil) + + <-wftCompleted + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) + if sticky { + s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) + } + + <-actCompleted + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) - _, err = s.pollWftAndHandle(tv, sticky, + s.pollWftAndHandle(tv, sticky, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) return respondCompleteWorkflow(tv, unpinned), nil }) - s.NoError(err) - s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), nil) + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) +} + +func transitionTo(d *deploymentpb.Deployment) *workflow.DeploymentTransition { + return &workflow.DeploymentTransition{ + Deployment: d, + } } func (s *Versioning3Suite) TestWorkflowWithPinnedOverride_Sticky() { @@ -229,47 +247,52 @@ func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { s.warmUpSticky(tv) } - override := pinnedOverride(tv.Deployment()) - we := s.startWorkflow(tv, override) - - _, err := s.pollWftAndHandle(tv, false, + wftCompleted := make(chan interface{}) + s.pollWftAndHandle(tv, false, wftCompleted, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) return respondWftWithActivity(tv, sticky, unpinned, "5"), nil }) - s.NoError(err) - s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), override) - if sticky { - s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) - } + s.waitForDeploymentDataPropagation(tv, tqTypeWf) - _, err = s.pollActivityAndHandle(tv, + actCompleted := make(chan interface{}) + s.pollActivityAndHandle(tv, actCompleted, func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { s.NotNil(task) return respondActivity(), nil }) - s.NoError(err) - s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), override) + s.waitForDeploymentDataPropagation(tv, tqTypeAct) + + override := pinnedOverride(tv.Deployment()) + we := s.startWorkflow(tv, override) + + <-wftCompleted + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + if sticky { + s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) + } - _, err = s.pollWftAndHandle(tv, sticky, + <-actCompleted + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + + s.pollWftAndHandle(tv, sticky, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) return respondCompleteWorkflow(tv, unpinned), nil }) - s.NoError(err) - s.verifyWorkflowVersioning(we, unpinned, tv.Deployment(), override) + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) } func (s *Versioning3Suite) updateTaskQueueDeploymentData( tv *testvars.TestVars, timeSinceCurrent time.Duration, + tqTypes ...enumspb.TaskQueueType, ) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) defer cancel() lastBecameCurrent := time.Now().Add(-timeSinceCurrent) - types := []enumspb.TaskQueueType{enumspb.TASK_QUEUE_TYPE_WORKFLOW, enumspb.TASK_QUEUE_TYPE_ACTIVITY} - for _, t := range types { + for _, t := range tqTypes { _, err := s.GetTestCluster().MatchingClient().SyncDeploymentUserData( ctx, &matchingservice.SyncDeploymentUserDataRequest{ NamespaceId: s.GetNamespaceID(s.Namespace()), @@ -284,69 +307,22 @@ func (s *Versioning3Suite) updateTaskQueueDeploymentData( ) s.NoError(err) } - s.waitForDeploymentDataPropagation(ctx, tv) -} - -func (s *Versioning3Suite) waitForDeploymentDataPropagation( - ctx context.Context, - tv *testvars.TestVars, -) { - v := s.GetTestCluster().Host().DcClient().GetValue(dynamicconfig.MatchingNumTaskqueueReadPartitions.Key()) - s.NotEmpty(v, "versioning tests require setting explicit number of partitions") - count, ok := v[0].Value.(int) - s.True(ok, "partition count is not an int") - partitionCount := count - - type partAndType struct { - part int - tp enumspb.TaskQueueType - } - remaining := make(map[partAndType]struct{}) - for i := 0; i < partitionCount; i++ { - remaining[partAndType{i, enumspb.TASK_QUEUE_TYPE_ACTIVITY}] = struct{}{} - remaining[partAndType{i, enumspb.TASK_QUEUE_TYPE_WORKFLOW}] = struct{}{} - } - nsId := s.GetNamespaceID(s.Namespace()) - f, err := tqid.NewTaskQueueFamily(nsId, tv.TaskQueue().GetName()) - deployment := tv.Deployment() - s.Eventually(func() bool { - for pt := range remaining { - s.NoError(err) - partition := f.TaskQueue(pt.tp).NormalPartition(pt.part) - // Use lower-level GetTaskQueueUserData instead of GetWorkerBuildIdCompatibility - // here so that we can target activity queues. - res, err := s.GetTestCluster().MatchingClient().GetTaskQueueUserData( - ctx, - &matchingservice.GetTaskQueueUserDataRequest{ - NamespaceId: nsId, - TaskQueue: partition.RpcName(), - TaskQueueType: partition.TaskType(), - }) - s.NoError(err) - perTypes := res.GetUserData().GetData().GetPerType() - if perTypes != nil { - deps := perTypes[int32(pt.tp)].GetDeploymentData().GetDeployments() - for _, d := range deps { - if d.GetDeployment().Equal(deployment) { - delete(remaining, pt) - } - } - } - } - return len(remaining) == 0 - }, 10*time.Second, 100*time.Millisecond) + s.waitForDeploymentDataPropagation(tv, tqTypes...) } func (s *Versioning3Suite) verifyWorkflowVersioning( - we *commonpb.WorkflowExecution, + tv *testvars.TestVars, behavior enumspb.VersioningBehavior, deployment *deploymentpb.Deployment, override *workflow.VersioningOverride, + transition *workflow.DeploymentTransition, ) { dwf, err := s.FrontendClient().DescribeWorkflowExecution( context.Background(), &workflowservice.DescribeWorkflowExecutionRequest{ Namespace: s.Namespace(), - Execution: we, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: tv.WorkflowID(), + }, }, ) s.NoError(err) @@ -367,6 +343,13 @@ func (s *Versioning3Suite) verifyWorkflowVersioning( actualOverrideDeployment, )) } + + if !versioningInfo.GetDeploymentTransition().Equal(transition) { + s.Fail(fmt.Sprintf("deployment override mismatch. expected: {%s}, actual: {%s}", + transition, + versioningInfo.GetDeploymentTransition(), + )) + } } func respondActivity() *workflowservice.RespondActivityTaskCompletedRequest { @@ -469,53 +452,79 @@ func (s *Versioning3Suite) startWorkflow( // do not grow larger that DB column limit (currently as low as 272 chars). func (s *Versioning3Suite) Name() string { fullName := s.T().Name() - if len(fullName) <= 300 { + if len(fullName) <= 30 { return fullName } return fmt.Sprintf("%s-%08x", - fullName[len(fullName)-25:], + fullName[len(fullName)-21:], farm.Fingerprint32([]byte(fullName)), ) } +// pollWftAndHandle can be used in sync and async mode. For async mode pass the async channel. It +// will be closed when the task is handled. func (s *Versioning3Suite) pollWftAndHandle( tv *testvars.TestVars, sticky bool, + async chan<- interface{}, handler func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error), -) (*workflowservice.RespondWorkflowTaskCompletedResponse, error) { +) { poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) d := tv.Deployment() tq := tv.TaskQueue() if sticky { tq = tv.StickyTaskQueue() } - return poller.PollWorkflowTask( - &workflowservice.PollWorkflowTaskQueueRequest{ - WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ - BuildId: d.BuildId, - DeploymentSeriesName: d.SeriesName, - UseVersioning: true, + f := func() { + _, err := poller.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{ + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: d.BuildId, + DeploymentSeriesName: d.SeriesName, + UseVersioning: true, + }, + TaskQueue: tq, }, - TaskQueue: tq, - }, - ).HandleTask(tv, handler) + ).HandleTask(tv, handler) + s.NoError(err) + } + if async == nil { + f() + } else { + go func() { + f() + close(async) + }() + } } func (s *Versioning3Suite) pollActivityAndHandle( tv *testvars.TestVars, + async chan<- interface{}, handler func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error), -) (*workflowservice.RespondActivityTaskCompletedResponse, error) { +) { poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) d := tv.Deployment() - return poller.PollActivityTask( - &workflowservice.PollActivityTaskQueueRequest{ - WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ - BuildId: d.BuildId, - DeploymentSeriesName: d.SeriesName, - UseVersioning: true, + f := func() { + _, err := poller.PollActivityTask( + &workflowservice.PollActivityTaskQueueRequest{ + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: d.BuildId, + DeploymentSeriesName: d.SeriesName, + UseVersioning: true, + }, }, - }, - ).HandleTask(tv, handler, taskpoller.WithTimeout(time.Minute)) + ).HandleTask(tv, handler, taskpoller.WithTimeout(time.Minute)) + s.NoError(err) + } + if async == nil { + f() + } else { + go func() { + f() + close(async) + }() + } } func (s *Versioning3Suite) idlePollWorkflow( @@ -603,3 +612,57 @@ func (s *Versioning3Suite) warmUpSticky( taskpoller.WithTimeout(minPollTime), ) } + +func (s *Versioning3Suite) waitForDeploymentDataPropagation( + tv *testvars.TestVars, + tqTypes ...enumspb.TaskQueueType, +) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + + v := s.GetTestCluster().Host().DcClient().GetValue(dynamicconfig.MatchingNumTaskqueueReadPartitions.Key()) + s.NotEmpty(v, "versioning tests require setting explicit number of partitions") + count, ok := v[0].Value.(int) + s.True(ok, "partition count is not an int") + partitionCount := count + + type partAndType struct { + part int + tp enumspb.TaskQueueType + } + remaining := make(map[partAndType]struct{}) + for i := 0; i < partitionCount; i++ { + for _, tqt := range tqTypes { + remaining[partAndType{i, tqt}] = struct{}{} + } + } + nsId := s.GetNamespaceID(s.Namespace()) + f, err := tqid.NewTaskQueueFamily(nsId, tv.TaskQueue().GetName()) + deployment := tv.Deployment() + s.Eventually(func() bool { + for pt := range remaining { + s.NoError(err) + partition := f.TaskQueue(pt.tp).NormalPartition(pt.part) + // Use lower-level GetTaskQueueUserData instead of GetWorkerBuildIdCompatibility + // here so that we can target activity queues. + res, err := s.GetTestCluster().MatchingClient().GetTaskQueueUserData( + ctx, + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: nsId, + TaskQueue: partition.RpcName(), + TaskQueueType: partition.TaskType(), + }) + s.NoError(err) + perTypes := res.GetUserData().GetData().GetPerType() + if perTypes != nil { + deps := perTypes[int32(pt.tp)].GetDeploymentData().GetDeployments() + for _, d := range deps { + if d.GetDeployment().Equal(deployment) { + delete(remaining, pt) + } + } + } + } + return len(remaining) == 0 + }, 10*time.Second, 100*time.Millisecond) +} From 41640513c801c873ae1d5f8cea078c5a8fedc499 Mon Sep 17 00:00:00 2001 From: ShahabT Date: Fri, 29 Nov 2024 13:32:50 -0800 Subject: [PATCH 04/21] transition tests --- service/matching/task_reader.go | 7 +- tests/versioning_3_test.go | 121 ++++++++++++++++++++++++-------- 2 files changed, 97 insertions(+), 31 deletions(-) diff --git a/service/matching/task_reader.go b/service/matching/task_reader.go index 7117d859a69..7ae5c805f09 100644 --- a/service/matching/task_reader.go +++ b/service/matching/task_reader.go @@ -39,6 +39,7 @@ import ( "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/primitives/timestamp" + serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/common/util" "go.temporal.io/server/internal/goro" ) @@ -155,10 +156,12 @@ dispatchLoop: continue dispatchLoop } + var stickyUnavailable *serviceerrors.StickyWorkerUnavailable // if task is still valid (truly valid or unable to verify if task is valid) metrics.BufferThrottlePerTaskQueueCounter.With(tr.taggedMetricsHandler()).Record(1) - if !errors.Is(err, context.DeadlineExceeded) && !errors.Is(err, context.Canceled) { - // Don't log here if encounters missing user data error when dispatch a versioned task. + if !errors.Is(err, context.DeadlineExceeded) && !errors.Is(err, context.Canceled) && + // StickyWorkerUnavailable is expected for versioned sticky queues + !errors.As(err, &stickyUnavailable) { tr.throttledLogger().Error("taskReader: unexpected error dispatching task", tag.Error(err)) } util.InterruptibleSleep(ctx, taskReaderOfferThrottleWait) diff --git a/tests/versioning_3_test.go b/tests/versioning_3_test.go index 0b2ee3a218e..ff95b494b63 100644 --- a/tests/versioning_3_test.go +++ b/tests/versioning_3_test.go @@ -156,25 +156,24 @@ func (s *Versioning3Suite) TestUnpinnedTask_OldDeployment() { ) } -func (s *Versioning3Suite) TestUnpinnedWorkflow_Sticky() { +func (s *Versioning3Suite) TestWorkflowWithPinnedOverride_Sticky() { s.RunTestWithMatchingBehavior( func() { - s.testUnpinnedWorkflow(true) + s.testWorkflowWithPinnedOverride(true) }, ) } -func (s *Versioning3Suite) TestUnpinnedWorkflow_NoSticky() { +func (s *Versioning3Suite) TestWorkflowWithPinnedOverride_NoSticky() { s.RunTestWithMatchingBehavior( func() { - s.testUnpinnedWorkflow(false) + s.testWorkflowWithPinnedOverride(false) }, ) } -func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { +func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { tv := testvars.New(s) - d := tv.Deployment() if sticky { s.warmUpSticky(tv) @@ -184,7 +183,6 @@ func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { s.pollWftAndHandle(tv, false, wftCompleted, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - s.verifyWorkflowVersioning(tv, unspecified, nil, nil, transitionTo(d)) return respondWftWithActivity(tv, sticky, unpinned, "5"), nil }) s.waitForDeploymentDataPropagation(tv, tqTypeWf) @@ -197,51 +195,45 @@ func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { }) s.waitForDeploymentDataPropagation(tv, tqTypeAct) - s.updateTaskQueueDeploymentData(tv, 0, tqTypeWf, tqTypeAct) - - we := s.startWorkflow(tv, nil) + override := pinnedOverride(tv.Deployment()) + we := s.startWorkflow(tv, override) <-wftCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) if sticky { s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) } <-actCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) s.pollWftAndHandle(tv, sticky, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) return respondCompleteWorkflow(tv, unpinned), nil }) - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) -} - -func transitionTo(d *deploymentpb.Deployment) *workflow.DeploymentTransition { - return &workflow.DeploymentTransition{ - Deployment: d, - } + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) } -func (s *Versioning3Suite) TestWorkflowWithPinnedOverride_Sticky() { +func (s *Versioning3Suite) TestUnpinnedWorkflow_Sticky() { s.RunTestWithMatchingBehavior( func() { - s.testWorkflowWithPinnedOverride(true) + s.testUnpinnedWorkflow(true) }, ) } -func (s *Versioning3Suite) TestWorkflowWithPinnedOverride_NoSticky() { +func (s *Versioning3Suite) TestUnpinnedWorkflow_NoSticky() { s.RunTestWithMatchingBehavior( func() { - s.testWorkflowWithPinnedOverride(false) + s.testUnpinnedWorkflow(false) }, ) } -func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { +func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { tv := testvars.New(s) + d := tv.Deployment() if sticky { s.warmUpSticky(tv) @@ -251,6 +243,7 @@ func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { s.pollWftAndHandle(tv, false, wftCompleted, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) + s.verifyWorkflowVersioning(tv, unspecified, nil, nil, transitionTo(d)) return respondWftWithActivity(tv, sticky, unpinned, "5"), nil }) s.waitForDeploymentDataPropagation(tv, tqTypeWf) @@ -263,24 +256,94 @@ func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { }) s.waitForDeploymentDataPropagation(tv, tqTypeAct) - override := pinnedOverride(tv.Deployment()) - we := s.startWorkflow(tv, override) + s.updateTaskQueueDeploymentData(tv, 0, tqTypeWf, tqTypeAct) + + we := s.startWorkflow(tv, nil) <-wftCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) if sticky { s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) } <-actCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) s.pollWftAndHandle(tv, sticky, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) return respondCompleteWorkflow(tv, unpinned), nil }) - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) +} + +func (s *Versioning3Suite) TestTransitionFromWft_Sticky() { + s.testTransitionFromWft(true) +} + +func (s *Versioning3Suite) TestTransitionFromWft_NoSticky() { + s.testTransitionFromWft(false) +} + +func (s *Versioning3Suite) testTransitionFromWft(sticky bool) { + // Wf runs one TWF and one AC on dA, then the second WFT is redirected to dB and + // transitions the wf with it. + + tvA := testvars.New(s).WithBuildId("A") + tvB := tvA.WithBuildId("B") + dA := tvA.Deployment() + dB := tvB.Deployment() + + if sticky { + s.warmUpSticky(tvA) + } + + wftCompleted := make(chan interface{}) + s.pollWftAndHandle(tvA, false, wftCompleted, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + s.verifyWorkflowVersioning(tvA, unspecified, nil, nil, transitionTo(dA)) + return respondWftWithActivity(tvA, sticky, unpinned, "5"), nil + }) + s.waitForDeploymentDataPropagation(tvA, tqTypeWf) + + actCompleted := make(chan interface{}) + s.pollActivityAndHandle(tvA, actCompleted, + func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { + s.NotNil(task) + return respondActivity(), nil + }) + s.waitForDeploymentDataPropagation(tvA, tqTypeAct) + + s.updateTaskQueueDeploymentData(tvA, 0, tqTypeWf, tqTypeAct) + + we := s.startWorkflow(tvA, nil) + + <-wftCompleted + s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + if sticky { + s.verifyWorkflowStickyQueue(we, tvA.StickyTaskQueue()) + } + + <-actCompleted + s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + + // Set B as the current deployment + s.updateTaskQueueDeploymentData(tvB, 0, tqTypeWf, tqTypeAct) + + s.pollWftAndHandle(tvB, false, nil, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, transitionTo(dB)) + return respondCompleteWorkflow(tvB, unpinned), nil + }) + s.verifyWorkflowVersioning(tvB, unpinned, dB, nil, nil) +} + +func transitionTo(d *deploymentpb.Deployment) *workflow.DeploymentTransition { + return &workflow.DeploymentTransition{ + Deployment: d, + } } func (s *Versioning3Suite) updateTaskQueueDeploymentData( From 009217b2ea8b02a07bea929d6e9b642a15ecef72 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 14:44:35 -0800 Subject: [PATCH 05/21] add form of test that would work with new sdk --- tests/deployment_test.go | 98 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 98 insertions(+) diff --git a/tests/deployment_test.go b/tests/deployment_test.go index 5bafe6e5e6d..ff576a704ad 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -28,6 +28,8 @@ import ( "context" "errors" "fmt" + "go.temporal.io/sdk/worker" + "go.temporal.io/sdk/workflow" "testing" "time" @@ -1010,6 +1012,102 @@ func (s *DeploymentSuite) checkListAndWaitForBatchCompletion(ctx context.Context } } +func (s *DeploymentSuite) waitForChan(ctx context.Context, ch chan struct{}) { + s.T().Helper() + select { + case <-ch: + case <-ctx.Done(): + s.FailNow("context timeout") + } +} + +func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_ChildWorkflow() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + deploymentA := &deploymentpb.Deployment{ + SeriesName: "seriesName", + BuildId: "A", + } + override := &workflowpb.VersioningOverride{ + Behavior: enumspb.VERSIONING_BEHAVIOR_PINNED, + Deployment: deploymentA, + } + pinnedOptsA := &workflowpb.WorkflowExecutionOptions{ + VersioningOverride: override, + } + tqName := "test-tq-child-override" + + // create deployment so that GetDeploymentReachability doesn't error + s.createDeploymentAndWaitForExist(deploymentA, &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}) + + // define parent and child worfklows + parentStarted := make(chan struct{}, 1) + childOverrideValidated := make(chan struct{}, 1) + child := func(cctx workflow.Context) (string, error) { + // no worker will take this, since we can't make a pinned worker with the old sdk + return "hello", nil + } + parent := func(ctx workflow.Context) error { + parentStarted <- struct{}{} + // after the test receives "parentStarted", we set the pinned override, and this workflow will + // make no more progress by itself, since we have no sdk workers that can handle this + //// wait for signal + //workflow.GetSignalChannel(ctx, "wait").Receive(ctx, nil) + // + //// check that parent's override is set + //parentWE := workflow.GetInfo(ctx).WorkflowExecution + //s.checkDescribeWorkflowAfterOverride( + // context.Background(), + // &commonpb.WorkflowExecution{WorkflowId: parentWE.ID, RunId: parentWE.RunID}, + // override) + // + //// run child workflow + //fut := workflow.ExecuteChildWorkflow(workflow.WithChildOptions(ctx, workflow.ChildWorkflowOptions{ + // TaskQueue: tqName, + //}), "child") + // + //// check that child's override is set + //var childWE workflow.Execution + //s.NoError(fut.GetChildWorkflowExecution().Get(ctx, &childWE)) + //s.checkDescribeWorkflowAfterOverride( + // context.Background(), + // &commonpb.WorkflowExecution{WorkflowId: childWE.ID, RunId: childWE.RunID}, + // override) + //childOverrideValidated <- struct{}{} + return nil + } + + unversionedWorker := worker.New(s.sdkClient, tqName, worker.Options{MaxConcurrentWorkflowTaskPollers: numPollers}) + unversionedWorker.RegisterWorkflowWithOptions(parent, workflow.RegisterOptions{Name: "parent"}) + unversionedWorker.RegisterWorkflowWithOptions(child, workflow.RegisterOptions{Name: "child"}) + s.NoError(unversionedWorker.Start()) + defer unversionedWorker.Stop() + + run, err := s.sdkClient.ExecuteWorkflow(ctx, sdkclient.StartWorkflowOptions{TaskQueue: tqName}, "parent") + s.NoError(err) + // wait for parent to start + s.waitForChan(ctx, parentStarted) + close(parentStarted) // force panic if replayed + + // set override on parent + updateResp, err := s.FrontendClient().UpdateWorkflowExecutionOptions(ctx, &workflowservice.UpdateWorkflowExecutionOptionsRequest{ + Namespace: s.Namespace(), + WorkflowExecution: &commonpb.WorkflowExecution{WorkflowId: run.GetID(), RunId: run.GetRunID()}, + WorkflowExecutionOptions: pinnedOptsA, + UpdateMask: &fieldmaskpb.FieldMask{Paths: []string{"versioning_override"}}, + }) + s.NoError(err) + s.True(proto.Equal(updateResp.GetWorkflowExecutionOptions(), pinnedOptsA)) + + // unblock the parent workflow so it will start its child + s.NoError(s.sdkClient.SignalWorkflow(ctx, run.GetID(), run.GetRunID(), "wait", nil)) + + // wait for child override to be validated (parent workflow might not complete, because no worker is polling + // that matches the child) + s.waitForChan(ctx, childOverrideValidated) + close(childOverrideValidated) // force panic if replayed +} + func (s *DeploymentSuite) TestStartWorkflowExecution_WithPinnedOverride() { ctx := context.Background() deploymentA := &deploymentpb.Deployment{ From e2d6fc23fcd4f4989d5ea392122318d442cdde37 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 15:29:11 -0800 Subject: [PATCH 06/21] another non-working child override inheritance test --- tests/child_workflow_test.go | 43 +++++++++++++++++++++++++++++++ tests/deployment_test.go | 49 ++++++++++++++++++------------------ 2 files changed, 68 insertions(+), 24 deletions(-) diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index 1e4810cae0e..5393662b662 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -25,15 +25,18 @@ package tests import ( + "context" "fmt" "sort" "testing" "time" "github.com/pborman/uuid" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" + deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" failurepb "go.temporal.io/api/failure/v1" filterpb "go.temporal.io/api/filter/v1" @@ -45,6 +48,7 @@ import ( "go.temporal.io/server/common/payload" "go.temporal.io/server/common/payloads" "go.temporal.io/server/tests/testcore" + "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/durationpb" "google.golang.org/protobuf/types/known/timestamppb" ) @@ -59,6 +63,39 @@ func TestChildWorkflowSuite(t *testing.T) { } func (s *ChildWorkflowSuite) TestChildWorkflowExecution() { + s.testChildWorkflowExecution(nil) +} + +func (s *ChildWorkflowSuite) TestChildWorkflowExecution_WithVersioningOverride() { + deploymentA := &deploymentpb.Deployment{ + SeriesName: "seriesName", + BuildId: "A", + } + override := &workflowpb.VersioningOverride{ + Behavior: enumspb.VERSIONING_BEHAVIOR_PINNED, + Deployment: deploymentA, + } + s.testChildWorkflowExecution(override) +} + +func (s *ChildWorkflowSuite) checkDescribeWorkflowAfterOverride( + wf *commonpb.WorkflowExecution, + expectedOverride *workflowpb.VersioningOverride, +) { + s.EventuallyWithT(func(t *assert.CollectT) { + a := assert.New(t) + resp, err := s.FrontendClient().DescribeWorkflowExecution(context.Background(), &workflowservice.DescribeWorkflowExecutionRequest{ + Namespace: s.Namespace(), + Execution: wf, + }) + a.NoError(err) + a.NotNil(resp) + a.NotNil(resp.GetWorkflowExecutionInfo()) + a.True(proto.Equal(expectedOverride, resp.GetWorkflowExecutionInfo().GetVersioningInfo().GetVersioningOverride())) + }, 5*time.Second, 50*time.Millisecond) +} + +func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.VersioningOverride) { parentID := "functional-child-workflow-test-parent" childID := "functional-child-workflow-test-child" grandchildID := "functional-child-workflow-test-grandchild" @@ -96,6 +133,7 @@ func (s *ChildWorkflowSuite) TestChildWorkflowExecution() { WorkflowRunTimeout: durationpb.New(100 * time.Second), WorkflowTaskTimeout: durationpb.New(1 * time.Second), Identity: identity, + VersioningOverride: override, } we, err0 := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) @@ -332,6 +370,9 @@ func (s *ChildWorkflowSuite) TestChildWorkflowExecution() { ) s.Equal(time.Duration(0), childStartedEventAttrs.GetWorkflowExecutionTimeout().AsDuration()) s.Equal(200*time.Second, childStartedEventAttrs.GetWorkflowRunTimeout().AsDuration()) + // check versioning override was inherited + s.ProtoEqual(override, childStartedEventAttrs.GetVersioningOverride()) + s.checkDescribeWorkflowAfterOverride(&commonpb.WorkflowExecution{WorkflowId: childID, RunId: childRunID}, override) // Process GrandchildExecution Started event and Process Grandchild Execution and complete it _, err = pollerChild.PollAndProcessWorkflowTask() @@ -354,6 +395,8 @@ func (s *ChildWorkflowSuite) TestChildWorkflowExecution() { s.NotNil(grandchildStartedEventAttrs.GetRootWorkflowExecution()) s.Equal(parentID, grandchildStartedEventAttrs.RootWorkflowExecution.GetWorkflowId()) s.Equal(we.GetRunId(), grandchildStartedEventAttrs.RootWorkflowExecution.GetRunId()) + // check versioning override was inherited + s.ProtoEqual(override, grandchildStartedEventAttrs.GetVersioningOverride()) // Process GrandchildExecution completed event and complete child execution _, err = pollerChild.PollAndProcessWorkflowTask() diff --git a/tests/deployment_test.go b/tests/deployment_test.go index ff576a704ad..c78eae57e37 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -1021,7 +1021,8 @@ func (s *DeploymentSuite) waitForChan(ctx context.Context, ch chan struct{}) { } } -func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_ChildWorkflow() { +func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_ChildWorkflowWithSDK() { + s.T().Skip("needs new sdk with deployment pollers to work") ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() deploymentA := &deploymentpb.Deployment{ @@ -1051,29 +1052,29 @@ func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_ChildWorkflow() { parentStarted <- struct{}{} // after the test receives "parentStarted", we set the pinned override, and this workflow will // make no more progress by itself, since we have no sdk workers that can handle this - //// wait for signal - //workflow.GetSignalChannel(ctx, "wait").Receive(ctx, nil) - // - //// check that parent's override is set - //parentWE := workflow.GetInfo(ctx).WorkflowExecution - //s.checkDescribeWorkflowAfterOverride( - // context.Background(), - // &commonpb.WorkflowExecution{WorkflowId: parentWE.ID, RunId: parentWE.RunID}, - // override) - // - //// run child workflow - //fut := workflow.ExecuteChildWorkflow(workflow.WithChildOptions(ctx, workflow.ChildWorkflowOptions{ - // TaskQueue: tqName, - //}), "child") - // - //// check that child's override is set - //var childWE workflow.Execution - //s.NoError(fut.GetChildWorkflowExecution().Get(ctx, &childWE)) - //s.checkDescribeWorkflowAfterOverride( - // context.Background(), - // &commonpb.WorkflowExecution{WorkflowId: childWE.ID, RunId: childWE.RunID}, - // override) - //childOverrideValidated <- struct{}{} + // wait for signal + workflow.GetSignalChannel(ctx, "wait").Receive(ctx, nil) + + // check that parent's override is set + parentWE := workflow.GetInfo(ctx).WorkflowExecution + s.checkDescribeWorkflowAfterOverride( + context.Background(), + &commonpb.WorkflowExecution{WorkflowId: parentWE.ID, RunId: parentWE.RunID}, + override) + + // run child workflow + fut := workflow.ExecuteChildWorkflow(workflow.WithChildOptions(ctx, workflow.ChildWorkflowOptions{ + TaskQueue: tqName, + }), "child") + + // check that child's override is set + var childWE workflow.Execution + s.NoError(fut.GetChildWorkflowExecution().Get(ctx, &childWE)) + s.checkDescribeWorkflowAfterOverride( + context.Background(), + &commonpb.WorkflowExecution{WorkflowId: childWE.ID, RunId: childWE.RunID}, + override) + childOverrideValidated <- struct{}{} return nil } From 0c172f1a3e920ecc1c869266eab26e1611d3ad48 Mon Sep 17 00:00:00 2001 From: ShahabT Date: Fri, 29 Nov 2024 15:37:01 -0800 Subject: [PATCH 07/21] fix lint --- tests/versioning_3_test.go | 171 +++++++++++++++++++++++++++++-------- 1 file changed, 134 insertions(+), 37 deletions(-) diff --git a/tests/versioning_3_test.go b/tests/versioning_3_test.go index ff95b494b63..cb032effc51 100644 --- a/tests/versioning_3_test.go +++ b/tests/versioning_3_test.go @@ -27,6 +27,7 @@ package tests import ( "context" "fmt" + "sync/atomic" "testing" "time" @@ -183,7 +184,7 @@ func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { s.pollWftAndHandle(tv, false, wftCompleted, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - return respondWftWithActivity(tv, sticky, unpinned, "5"), nil + return respondWftWithActivities(tv, sticky, unpinned, "5"), nil }) s.waitForDeploymentDataPropagation(tv, tqTypeWf) @@ -244,7 +245,7 @@ func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) s.verifyWorkflowVersioning(tv, unspecified, nil, nil, transitionTo(d)) - return respondWftWithActivity(tv, sticky, unpinned, "5"), nil + return respondWftWithActivities(tv, sticky, unpinned, "5"), nil }) s.waitForDeploymentDataPropagation(tv, tqTypeWf) @@ -298,43 +299,116 @@ func (s *Versioning3Suite) testTransitionFromWft(sticky bool) { s.warmUpSticky(tvA) } - wftCompleted := make(chan interface{}) - s.pollWftAndHandle(tvA, false, wftCompleted, + s.updateTaskQueueDeploymentData(tvA, 0, tqTypeWf, tqTypeAct) + we := s.startWorkflow(tvA, nil) + + s.pollWftAndHandle(tvA, false, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) s.verifyWorkflowVersioning(tvA, unspecified, nil, nil, transitionTo(dA)) - return respondWftWithActivity(tvA, sticky, unpinned, "5"), nil + return respondWftWithActivities(tvA, sticky, unpinned, "5"), nil }) - s.waitForDeploymentDataPropagation(tvA, tqTypeWf) + s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + if sticky { + s.verifyWorkflowStickyQueue(we, tvA.StickyTaskQueue()) + } - actCompleted := make(chan interface{}) - s.pollActivityAndHandle(tvA, actCompleted, + s.pollActivityAndHandle(tvA, nil, func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { s.NotNil(task) return respondActivity(), nil }) - s.waitForDeploymentDataPropagation(tvA, tqTypeAct) + s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) - s.updateTaskQueueDeploymentData(tvA, 0, tqTypeWf, tqTypeAct) + // Set B as the current deployment + s.updateTaskQueueDeploymentData(tvB, 0, tqTypeWf, tqTypeAct) + + s.pollWftAndHandle(tvB, false, nil, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, transitionTo(dB)) + return respondCompleteWorkflow(tvB, unpinned), nil + }) + s.verifyWorkflowVersioning(tvB, unpinned, dB, nil, nil) +} +func (s *Versioning3Suite) TestTransitionFromActivity_Sticky() { + s.testTransitionFromActivity(true) +} + +func (s *Versioning3Suite) TestTransitionFromActivity_NoSticky() { + s.testTransitionFromActivity(false) +} + +func (s *Versioning3Suite) testTransitionFromActivity(sticky bool) { + // Wf runs one TWF on dA and schedules four activities, then: + // 1. The first and second activities starts on dA + // 2. Current deployment becomes dB + // 3. The third activity is redirected to dB and starts a transition in the wf, without being + // dispatched. + // 4. The 4th activity also does not start on any of the builds although there are pending + // pollers on both. + // 5. The transition generates a WFT and it is started in dB. + // 6. The 1st act is completed here while the transition is going on. + // 7. The 2nd act fails and makes another attempt. But it is not dispatched. + // 8. WFT completes and the transition completes. + // 9. All the 3 remaining activities are now dispatched and completed. + + tvA := testvars.New(s).WithBuildId("A") + tvB := tvA.WithBuildId("B") + dA := tvA.Deployment() + dB := tvB.Deployment() + + if sticky { + s.warmUpSticky(tvA) + } + + s.updateTaskQueueDeploymentData(tvA, 0, tqTypeWf, tqTypeAct) we := s.startWorkflow(tvA, nil) - <-wftCompleted + s.pollWftAndHandle(tvA, false, nil, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) + s.verifyWorkflowVersioning(tvA, unspecified, nil, nil, transitionTo(dA)) + return respondWftWithActivities(tvA, sticky, unpinned, "5", "6", "7", "8"), nil + }) s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) if sticky { s.verifyWorkflowStickyQueue(we, tvA.StickyTaskQueue()) } - <-actCompleted - s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) - // Set B as the current deployment s.updateTaskQueueDeploymentData(tvB, 0, tqTypeWf, tqTypeAct) + // The poller should be present to the activity task is redirected, but it should not receive a + // task until transition completes in the next wft. + transitionCompleted := atomic.Bool{} + actCompleted := make(chan interface{}) + s.pollActivityAndHandle(tvB, actCompleted, + func(task *workflowservice.PollActivityTaskQueueResponse) (*workflowservice.RespondActivityTaskCompletedRequest, error) { + // Activity should not start until the transition is completed + s.True(transitionCompleted.Load()) + s.NotNil(task) + return respondActivity(), nil + }) + s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + + // The transition should create a new WFT to be sent to dB. Poller responds with empty wft complete. s.pollWftAndHandle(tvB, false, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, transitionTo(dB)) + transitionCompleted.Store(true) + return respondEmptyWft(tvB, sticky, unpinned), nil + }) + s.verifyWorkflowVersioning(tvB, unpinned, dB, nil, nil) + if sticky { + s.verifyWorkflowStickyQueue(we, tvB.StickyTaskQueue()) + } + + s.pollWftAndHandle(tvB, false, nil, + func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { + s.NotNil(task) return respondCompleteWorkflow(tvB, unpinned), nil }) s.verifyWorkflowVersioning(tvB, unpinned, dB, nil, nil) @@ -419,11 +493,11 @@ func respondActivity() *workflowservice.RespondActivityTaskCompletedRequest { return &workflowservice.RespondActivityTaskCompletedRequest{} } -func respondWftWithActivity( +func respondWftWithActivities( tv *testvars.TestVars, sticky bool, behavior enumspb.VersioningBehavior, - activityId string, + activityIds ...string, ) *workflowservice.RespondWorkflowTaskCompletedRequest { var stickyAttr *taskqueuepb.StickyExecutionAttributes if sticky { @@ -432,26 +506,49 @@ func respondWftWithActivity( ScheduleToStartTimeout: durationpb.New(5 * time.Second), } } - return &workflowservice.RespondWorkflowTaskCompletedRequest{ - Commands: []*commandpb.Command{ - { - CommandType: enumspb.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK, - Attributes: &commandpb.Command_ScheduleActivityTaskCommandAttributes{ - ScheduleActivityTaskCommandAttributes: &commandpb.ScheduleActivityTaskCommandAttributes{ - ActivityId: activityId, - ActivityType: &commonpb.ActivityType{Name: "act"}, - TaskQueue: tv.TaskQueue(), - Input: payloads.EncodeString("input"), - // TODO (shahab): tests with forced task forward take multiple seconds. Need to know why? - ScheduleToCloseTimeout: durationpb.New(10 * time.Second), - ScheduleToStartTimeout: durationpb.New(10 * time.Second), - StartToCloseTimeout: durationpb.New(1 * time.Second), - HeartbeatTimeout: durationpb.New(1 * time.Second), - RequestEagerExecution: false, - }, + var commands []*commandpb.Command + for _, a := range activityIds { + commands = append(commands, &commandpb.Command{ + CommandType: enumspb.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK, + Attributes: &commandpb.Command_ScheduleActivityTaskCommandAttributes{ + ScheduleActivityTaskCommandAttributes: &commandpb.ScheduleActivityTaskCommandAttributes{ + ActivityId: a, + ActivityType: &commonpb.ActivityType{Name: "act"}, + TaskQueue: tv.TaskQueue(), + Input: payloads.EncodeString("input"), + // TODO (shahab): tests with forced task forward take multiple seconds. Need to know why? + ScheduleToCloseTimeout: durationpb.New(10 * time.Second), + ScheduleToStartTimeout: durationpb.New(10 * time.Second), + StartToCloseTimeout: durationpb.New(1 * time.Second), + HeartbeatTimeout: durationpb.New(1 * time.Second), + RequestEagerExecution: false, }, }, - }, + }) + } + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + StickyAttributes: stickyAttr, + ReturnNewWorkflowTask: false, + ForceCreateNewWorkflowTask: false, + Deployment: tv.Deployment(), + VersioningBehavior: behavior, + } +} + +func respondEmptyWft( + tv *testvars.TestVars, + sticky bool, + behavior enumspb.VersioningBehavior, +) *workflowservice.RespondWorkflowTaskCompletedRequest { + var stickyAttr *taskqueuepb.StickyExecutionAttributes + if sticky { + stickyAttr = &taskqueuepb.StickyExecutionAttributes{ + WorkerTaskQueue: tv.StickyTaskQueue(), + ScheduleToStartTimeout: durationpb.New(5 * time.Second), + } + } + return &workflowservice.RespondWorkflowTaskCompletedRequest{ StickyAttributes: stickyAttr, ReturnNewWorkflowTask: false, ForceCreateNewWorkflowTask: false, @@ -598,7 +695,7 @@ func (s *Versioning3Suite) idlePollWorkflow( ) { poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) d := tv.Deployment() - poller.PollWorkflowTask( + _, _ = poller.PollWorkflowTask( &workflowservice.PollWorkflowTaskQueueRequest{ WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ BuildId: d.BuildId, @@ -624,7 +721,7 @@ func (s *Versioning3Suite) idlePollActivity( ) { poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) d := tv.Deployment() - poller.PollActivityTask( + _, _ = poller.PollActivityTask( &workflowservice.PollActivityTaskQueueRequest{ WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ BuildId: d.BuildId, @@ -662,7 +759,7 @@ func (s *Versioning3Suite) warmUpSticky( tv *testvars.TestVars, ) { poller := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) - poller.PollWorkflowTask( + _, _ = poller.PollWorkflowTask( &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: tv.StickyTaskQueue(), }, From e6ec3df292b2ebef9260e7ed44900e1bd031b00d Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 16:26:16 -0800 Subject: [PATCH 08/21] convert test to new poller, got 'history is nil' error --- tests/child_workflow_test.go | 152 +++++++++++++++++++++-------------- 1 file changed, 93 insertions(+), 59 deletions(-) diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index 5393662b662..c7fd75d19ee 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -27,6 +27,8 @@ package tests import ( "context" "fmt" + "go.temporal.io/server/common/testing/taskpoller" + "go.temporal.io/server/common/testing/testvars" "sort" "testing" "time" @@ -96,6 +98,13 @@ func (s *ChildWorkflowSuite) checkDescribeWorkflowAfterOverride( } func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.VersioningOverride) { + tv := testvars.New(s.T()) + var overrideDeployment *deploymentpb.Deployment + var overrideBehavior enumspb.VersioningBehavior + if override != nil { + overrideDeployment = override.GetDeployment() + overrideBehavior = override.GetBehavior() + } parentID := "functional-child-workflow-test-parent" childID := "functional-child-workflow-test-child" grandchildID := "functional-child-workflow-test-grandchild" @@ -163,7 +172,7 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver // Parent workflow logic wtHandlerParent := func( - task *workflowservice.PollWorkflowTaskQueueResponse) ([]*commandpb.Command, error) { + task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.Logger.Info("Processing workflow task for Parent", tag.WorkflowID(task.WorkflowExecution.WorkflowId)) parentStartedEvent = task.History.Events[0] @@ -172,7 +181,7 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.Logger.Info("Starting child execution") childExecutionStarted = true - return []*commandpb.Command{{ + commands := []*commandpb.Command{{ CommandType: enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION, Attributes: &commandpb.Command_StartChildWorkflowExecutionCommandAttributes{ StartChildWorkflowExecutionCommandAttributes: &commandpb.StartChildWorkflowExecutionCommandAttributes{ @@ -188,24 +197,46 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver SearchAttributes: searchAttr, }, }, - }}, nil + }} + + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + Identity: identity, + Namespace: s.Namespace(), + Deployment: overrideDeployment, + VersioningBehavior: overrideBehavior, + }, nil } else if task.PreviousStartedEventId > 0 { for _, event := range task.History.Events[task.PreviousStartedEventId:] { if event.GetEventType() == enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED { childStartedEventFromParent = event - return []*commandpb.Command{}, nil + commands := []*commandpb.Command{} + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + Identity: identity, + Namespace: s.Namespace(), + Deployment: overrideDeployment, + VersioningBehavior: overrideBehavior, + }, nil } if event.GetEventType() == enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED { childCompletedEventFromParent = event - return []*commandpb.Command{{ + commands := []*commandpb.Command{{ CommandType: enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION, Attributes: &commandpb.Command_CompleteWorkflowExecutionCommandAttributes{ CompleteWorkflowExecutionCommandAttributes: &commandpb.CompleteWorkflowExecutionCommandAttributes{ Result: payloads.EncodeString("Done"), }, }, - }}, nil + }} + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + Identity: identity, + Namespace: s.Namespace(), + Deployment: overrideDeployment, + VersioningBehavior: overrideBehavior, + }, nil } } } @@ -217,7 +248,7 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver var childStartedEvent *historypb.HistoryEvent var childRunID string // Child workflow logic - wtHandlerChild := func(task *workflowservice.PollWorkflowTaskQueueResponse) ([]*commandpb.Command, error) { + wtHandlerChild := func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { if task.PreviousStartedEventId <= 0 { childStartedEvent = task.History.Events[0] childRunID = task.WorkflowExecution.GetRunId() @@ -228,7 +259,7 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.Logger.Info("Starting grandchild execution") grandchildExecutionStarted = true - return []*commandpb.Command{{ + commands := []*commandpb.Command{{ CommandType: enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION, Attributes: &commandpb.Command_StartChildWorkflowExecutionCommandAttributes{ StartChildWorkflowExecutionCommandAttributes: &commandpb.StartChildWorkflowExecutionCommandAttributes{ @@ -244,24 +275,46 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver SearchAttributes: searchAttr, }, }, - }}, nil + }} + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + Identity: identity, + Namespace: s.Namespace(), + Deployment: overrideDeployment, + VersioningBehavior: overrideBehavior, + }, nil } if task.PreviousStartedEventId > 0 { for _, event := range task.History.Events[task.PreviousStartedEventId:] { if event.GetEventType() == enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED { - return []*commandpb.Command{}, nil + commands := []*commandpb.Command{} + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + Identity: identity, + Namespace: s.Namespace(), + Deployment: overrideDeployment, + VersioningBehavior: overrideBehavior, + }, nil + } if event.GetEventType() == enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED { childComplete = true - return []*commandpb.Command{{ + commands := []*commandpb.Command{{ CommandType: enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION, Attributes: &commandpb.Command_CompleteWorkflowExecutionCommandAttributes{ CompleteWorkflowExecutionCommandAttributes: &commandpb.CompleteWorkflowExecutionCommandAttributes{ Result: payloads.EncodeString("Child Done"), }, }, - }}, nil + }} + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + Identity: identity, + Namespace: s.Namespace(), + Deployment: overrideDeployment, + VersioningBehavior: overrideBehavior, + }, nil } } } @@ -272,56 +325,37 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver var grandchildStartedEvent *historypb.HistoryEvent // Grandchild workflow logic to check root workflow execution is carried correctly wtHandlerGrandchild := func( - task *workflowservice.PollWorkflowTaskQueueResponse) ([]*commandpb.Command, error) { + task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { if task.PreviousStartedEventId <= 0 { grandchildStartedEvent = task.History.Events[0] } s.Logger.Info("Processing workflow task for Grandchild", tag.WorkflowID(task.WorkflowExecution.WorkflowId)) grandchildComplete = true - return []*commandpb.Command{{ + commands := []*commandpb.Command{{ CommandType: enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION, Attributes: &commandpb.Command_CompleteWorkflowExecutionCommandAttributes{ CompleteWorkflowExecutionCommandAttributes: &commandpb.CompleteWorkflowExecutionCommandAttributes{ Result: payloads.EncodeString("Grandchild Done"), }, }, - }}, nil - } - - pollerParent := &testcore.TaskPoller{ - Client: s.FrontendClient(), - Namespace: s.Namespace(), - TaskQueue: taskQueueParent, - Identity: identity, - WorkflowTaskHandler: wtHandlerParent, - Logger: s.Logger, - T: s.T(), - } - - pollerChild := &testcore.TaskPoller{ - Client: s.FrontendClient(), - Namespace: s.Namespace(), - TaskQueue: taskQueueChild, - Identity: identity, - WorkflowTaskHandler: wtHandlerChild, - Logger: s.Logger, - T: s.T(), + }} + return &workflowservice.RespondWorkflowTaskCompletedRequest{ + Commands: commands, + Identity: identity, + Namespace: s.Namespace(), + Deployment: overrideDeployment, + VersioningBehavior: overrideBehavior, + }, nil } - pollerGrandchild := &testcore.TaskPoller{ - Client: s.FrontendClient(), - Namespace: s.Namespace(), - TaskQueue: taskQueueGrandchild, - Identity: identity, - WorkflowTaskHandler: wtHandlerGrandchild, - Logger: s.Logger, - T: s.T(), - } + pollerParent := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) // taskQueueParent + pollerChild := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) // taskQueueChild + pollerGrandchild := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) // taskQueueGrandchild // Make first workflow task to start child execution - _, err := pollerParent.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + _, err := pollerParent.PollAndHandleWorkflowTask(tv, wtHandlerParent) + s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(childExecutionStarted) s.NotNil(parentStartedEvent) @@ -331,13 +365,13 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.Nil(parentStartedEventAttrs.GetRootWorkflowExecution()) // Process ChildExecution Started event and Process Child Execution and complete it - _, err = pollerParent.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + _, err = pollerParent.PollAndHandleWorkflowTask(tv, wtHandlerParent) + s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) // Process Child workflow to start grandchild execution - _, err = pollerChild.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + _, err = pollerChild.PollAndHandleWorkflowTask(tv, wtHandlerChild) + s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.NotNil(childStartedEventFromParent) s.NotNil(childStartedEvent) @@ -375,13 +409,13 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.checkDescribeWorkflowAfterOverride(&commonpb.WorkflowExecution{WorkflowId: childID, RunId: childRunID}, override) // Process GrandchildExecution Started event and Process Grandchild Execution and complete it - _, err = pollerChild.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + _, err = pollerChild.PollAndHandleWorkflowTask(tv, wtHandlerChild) + s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) // Process Grandchild workflow - _, err = pollerGrandchild.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + _, err = pollerGrandchild.PollAndHandleWorkflowTask(tv, wtHandlerGrandchild) + s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(grandchildComplete) s.NotNil(grandchildStartedEvent) @@ -399,14 +433,14 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.ProtoEqual(override, grandchildStartedEventAttrs.GetVersioningOverride()) // Process GrandchildExecution completed event and complete child execution - _, err = pollerChild.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + _, err = pollerChild.PollAndHandleWorkflowTask(tv, wtHandlerChild) + s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(childComplete) // Process ChildExecution completed event and complete parent execution - _, err = pollerParent.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + _, err = pollerParent.PollAndHandleWorkflowTask(tv, wtHandlerParent) + s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.NotNil(childCompletedEventFromParent) completedAttributes := childCompletedEventFromParent.GetChildWorkflowExecutionCompletedEventAttributes() @@ -776,7 +810,7 @@ func (s *ChildWorkflowSuite) TestRetryChildWorkflowExecution() { // Process ChildExecution Started event _, err = pollerParent.PollAndProcessWorkflowTask() - s.Logger.Info("PollAndProcessWorkflowTask", tag.Error(err)) + s.Logger.Info("PollAndHWorkflowTask", tag.Error(err)) s.NoError(err) s.NotNil(startedEvent) From 3bf7cbd49af99efb87a894e1ee63b2b41b345665 Mon Sep 17 00:00:00 2001 From: ShahabT Date: Fri, 29 Nov 2024 16:33:01 -0800 Subject: [PATCH 09/21] fix user data manage tests --- .../matching/physical_task_queue_manager.go | 42 +--------- service/matching/user_data_manager_test.go | 80 ++++++++++++++++++- 2 files changed, 80 insertions(+), 42 deletions(-) diff --git a/service/matching/physical_task_queue_manager.go b/service/matching/physical_task_queue_manager.go index 5d59110ca5b..9e9c31991c2 100644 --- a/service/matching/physical_task_queue_manager.go +++ b/service/matching/physical_task_queue_manager.go @@ -39,7 +39,6 @@ import ( "go.temporal.io/api/serviceerror" taskqueuepb "go.temporal.io/api/taskqueue/v1" "go.temporal.io/api/workflowservice/v1" - "go.temporal.io/server/api/enums/v1" "go.temporal.io/server/api/matchingservice/v1" persistencespb "go.temporal.io/server/api/persistence/v1" taskqueuespb "go.temporal.io/server/api/taskqueue/v1" @@ -321,15 +320,6 @@ func (c *physicalTaskQueueManagerImpl) WaitUntilInitialized(ctx context.Context) func (c *physicalTaskQueueManagerImpl) SpoolTask(taskInfo *persistencespb.TaskInfo) error { c.liveness.markAlive() - if c.queue.version.Deployment() != nil { - fmt.Printf( - "\n %s shahab> spooled task %s %s %s\n\n", - time.Now(), - c.queue.partition.RpcName(), - c.queue.partition.TaskType().String(), - c.queue.version.Deployment(), - ) - } return c.backlogMgr.SpoolTask(taskInfo) } @@ -341,15 +331,6 @@ func (c *physicalTaskQueueManagerImpl) PollTask( ctx context.Context, pollMetadata *pollMetadata, ) (*internalTask, error) { - if c.queue.version.Deployment() != nil { - fmt.Printf( - "\n %s shahab> polling task %s %s %s\n\n", - time.Now(), - c.queue.partition.RpcName(), - c.queue.partition.TaskType().String(), - c.queue.version.Deployment(), - ) - } c.liveness.markAlive() c.currentPolls.Add(1) @@ -401,17 +382,6 @@ func (c *physicalTaskQueueManagerImpl) PollTask( (!task.isStarted() || !task.started.hasEmptyResponse()) { // Need to filter out the empty "started" ones c.tasksDispatchedInIntervals.incrementTaskCount() } - - if c.queue.version.Deployment() != nil { - fmt.Printf( - "\n %s shahab> polled task %s %s %s\n\n", - time.Now(), - c.queue.partition.RpcName(), - c.queue.partition.TaskType().String(), - c.queue.version.Deployment(), - ) - } - return task, nil } } @@ -566,17 +536,7 @@ func (c *physicalTaskQueueManagerImpl) TrySyncMatch(ctx context.Context, task *i childCtx, cancel := newChildContext(ctx, c.config.SyncMatchWaitDuration(), time.Second) defer cancel() - a, b := c.matcher.Offer(childCtx, task) - if task.source == enums.TASK_SOURCE_HISTORY && a && c.queue.version.Deployment() != nil { - fmt.Printf( - "\n %s shahab> sync-matched task %s %s %s\n\n", - time.Now(), - c.queue.partition.RpcName(), - c.queue.partition.TaskType().String(), - c.queue.version.Deployment(), - ) - } - return a, b + return c.matcher.Offer(childCtx, task) } func (c *physicalTaskQueueManagerImpl) ensureRegisteredInDeployment( diff --git a/service/matching/user_data_manager_test.go b/service/matching/user_data_manager_test.go index 27ab3173e6b..b7d91ac92be 100644 --- a/service/matching/user_data_manager_test.go +++ b/service/matching/user_data_manager_test.go @@ -153,12 +153,25 @@ func TestUserData_FetchesOnInit(t *testing.T) { TaskQueue: defaultRootTqID, TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, LastKnownUserDataVersion: 0, - WaitNewData: false, // first fetch is not long poll + WaitNewData: false, // first is not long poll }). Return(&matchingservice.GetTaskQueueUserDataResponse{ UserData: data1, }, nil) + tqCfg.matchingClientMock.EXPECT().GetTaskQueueUserData( + gomock.Any(), + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: defaultNamespaceId, + TaskQueue: defaultRootTqID, + TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, + LastKnownUserDataVersion: 1, + WaitNewData: true, // second is long poll + }). + Return(&matchingservice.GetTaskQueueUserDataResponse{ + UserData: data1, + }, nil).MaxTimes(3) + m := createUserDataManager(t, controller, tqCfg) m.config.GetUserDataMinWaitTime = 10 * time.Second // only one fetch @@ -283,6 +296,19 @@ func TestUserData_RetriesFetchOnUnavailable(t *testing.T) { }, nil }) + tqCfg.matchingClientMock.EXPECT().GetTaskQueueUserData( + gomock.Any(), + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: defaultNamespaceId, + TaskQueue: defaultRootTqID, + TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, + LastKnownUserDataVersion: 1, + WaitNewData: true, // after first successful poll, there would be long polls + }). + Return(&matchingservice.GetTaskQueueUserDataResponse{ + UserData: data1, + }, nil).MaxTimes(3) + m := createUserDataManager(t, controller, tqCfg) m.config.GetUserDataMinWaitTime = 10 * time.Second // wait on success m.config.GetUserDataRetryPolicy = backoff.NewExponentialRetryPolicy(50 * time.Millisecond). @@ -354,6 +380,19 @@ func TestUserData_RetriesFetchOnUnImplemented(t *testing.T) { }, nil }) + tqCfg.matchingClientMock.EXPECT().GetTaskQueueUserData( + gomock.Any(), + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: defaultNamespaceId, + TaskQueue: defaultRootTqID, + TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, + LastKnownUserDataVersion: 1, + WaitNewData: true, // after first successful poll, there would be long polls + }). + Return(&matchingservice.GetTaskQueueUserDataResponse{ + UserData: data1, + }, nil).MaxTimes(3) + m := createUserDataManager(t, controller, tqCfg) m.config.GetUserDataMinWaitTime = 10 * time.Second // wait on success m.config.GetUserDataRetryPolicy = backoff.NewExponentialRetryPolicy(50 * time.Millisecond). @@ -410,6 +449,19 @@ func TestUserData_FetchesUpTree(t *testing.T) { UserData: data1, }, nil) + tqCfg.matchingClientMock.EXPECT().GetTaskQueueUserData( + gomock.Any(), + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: defaultNamespaceId, + TaskQueue: taskQueue.NormalPartition(10).RpcName(), + TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, + LastKnownUserDataVersion: 1, + WaitNewData: true, // after first successful poll, there would be long polls + }). + Return(&matchingservice.GetTaskQueueUserDataResponse{ + UserData: data1, + }, nil).MaxTimes(3) + m := createUserDataManager(t, controller, tqCfg) m.config.GetUserDataMinWaitTime = 10 * time.Second // wait on success m.Start() @@ -448,6 +500,19 @@ func TestUserData_FetchesActivityToWorkflow(t *testing.T) { UserData: data1, }, nil) + tqCfg.matchingClientMock.EXPECT().GetTaskQueueUserData( + gomock.Any(), + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: defaultNamespaceId, + TaskQueue: defaultRootTqID, + TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, + LastKnownUserDataVersion: 1, + WaitNewData: true, // after first successful poll, there would be long polls + }). + Return(&matchingservice.GetTaskQueueUserDataResponse{ + UserData: data1, + }, nil).MaxTimes(3) + m := createUserDataManager(t, controller, tqCfg) m.config.GetUserDataMinWaitTime = 10 * time.Second // wait on success m.Start() @@ -490,6 +555,19 @@ func TestUserData_FetchesStickyToNormal(t *testing.T) { UserData: data1, }, nil) + tqCfg.matchingClientMock.EXPECT().GetTaskQueueUserData( + gomock.Any(), + &matchingservice.GetTaskQueueUserDataRequest{ + NamespaceId: defaultNamespaceId, + TaskQueue: normalName, + TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, + LastKnownUserDataVersion: 1, + WaitNewData: true, // after first successful poll, there would be long polls + }). + Return(&matchingservice.GetTaskQueueUserDataResponse{ + UserData: data1, + }, nil).MaxTimes(3) + m := createUserDataManager(t, controller, tqCfg) m.config.GetUserDataMinWaitTime = 10 * time.Second // wait on success m.Start() From b217ae66505f2cdef181d96731857a356ae94464 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 16:41:26 -0800 Subject: [PATCH 10/21] add print and rename tests --- common/testing/taskpoller/taskpoller.go | 1 + tests/child_workflow_test.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/common/testing/taskpoller/taskpoller.go b/common/testing/taskpoller/taskpoller.go index 8b77a9370c8..3330688ab88 100644 --- a/common/testing/taskpoller/taskpoller.go +++ b/common/testing/taskpoller/taskpoller.go @@ -233,6 +233,7 @@ func (p *workflowTaskPoller) pollTask( } var events []*historypb.HistoryEvent + fmt.Printf("\n\n\nCARLY DEBUG resp: %+v\n\n\n", resp) history := resp.History if history == nil { return nil, errors.New("history is nil") diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index c7fd75d19ee..3b6db11c1b9 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -64,7 +64,7 @@ func TestChildWorkflowSuite(t *testing.T) { suite.Run(t, new(ChildWorkflowSuite)) } -func (s *ChildWorkflowSuite) TestChildWorkflowExecution() { +func (s *ChildWorkflowSuite) TestChildWorkflowExecution_NoOverride() { s.testChildWorkflowExecution(nil) } From 9d6b5736570033da56f1aef6c2243812f27d8cdb Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 16:59:53 -0800 Subject: [PATCH 11/21] wrap test variables in tv.String --- tests/child_workflow_test.go | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index 3b6db11c1b9..c2abc91730b 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -105,17 +105,17 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver overrideDeployment = override.GetDeployment() overrideBehavior = override.GetBehavior() } - parentID := "functional-child-workflow-test-parent" - childID := "functional-child-workflow-test-child" - grandchildID := "functional-child-workflow-test-grandchild" - wtParent := "functional-child-workflow-test-parent-type" - wtChild := "functional-child-workflow-test-child-type" - wtGrandchild := "functional-child-workflow-test-grandchild-type" - tlParent := "functional-child-workflow-test-parent-taskqueue" - tlChild := "functional-child-workflow-test-child-taskqueue" - tlGrandchild := "functional-child-workflow-test-grandchild-taskqueue" - identity := "worker1" - saName := "CustomKeywordField" + parentID := tv.String("functional-child-workflow-test-parent") + childID := tv.String("functional-child-workflow-test-child") + grandchildID := tv.String("functional-child-workflow-test-grandchild") + wtParent := tv.String("functional-child-workflow-test-parent-type") + wtChild := tv.String("functional-child-workflow-test-child-type") + wtGrandchild := tv.String("functional-child-workflow-test-grandchild-type") + tlParent := tv.String("functional-child-workflow-test-parent-taskqueue") + tlChild := tv.String("functional-child-workflow-test-child-taskqueue") + tlGrandchild := tv.String("functional-child-workflow-test-grandchild-taskqueue") + identity := tv.String("worker1") + saName := tv.String("CustomKeywordField") // Uncomment this line to test with mapper. // saName = "AliasForCustomKeywordField" From 05849672e5fb481417b73460f17ef85baac7a4a1 Mon Sep 17 00:00:00 2001 From: David Reiss Date: Fri, 29 Nov 2024 20:00:41 -0500 Subject: [PATCH 12/21] make RandomizeString shorter --- tests/testcore/functional_test_base.go | 6 +++--- tests/testcore/utils.go | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/testcore/functional_test_base.go b/tests/testcore/functional_test_base.go index 1d70b7385b6..95d640cfaca 100644 --- a/tests/testcore/functional_test_base.go +++ b/tests/testcore/functional_test_base.go @@ -184,14 +184,14 @@ func (s *FunctionalTestBase) SetupSuite(defaultClusterConfigFile string, options s.operatorClient = s.testCluster.OperatorClient() s.httpAPIAddress = cluster.Host().FrontendHTTPAddress() - s.namespace = "functional-test-namespace" + s.namespace = RandomizeStr("namespace") s.Require().NoError(s.registerNamespaceWithDefaults(s.namespace)) - s.foreignNamespace = RandomizeStr("functional-foreign-test-namespace") + s.foreignNamespace = RandomizeStr("foreign-namespace") s.Require().NoError(s.registerNamespaceWithDefaults(s.foreignNamespace)) if clusterConfig.EnableArchival { - s.archivalNamespace = RandomizeStr("functional-archival-enabled-namespace") + s.archivalNamespace = RandomizeStr("archival-enabled-namespace") s.Require().NoError(s.registerArchivalNamespace(s.archivalNamespace)) } } diff --git a/tests/testcore/utils.go b/tests/testcore/utils.go index 38a6a24ac24..bd0fae1c14f 100644 --- a/tests/testcore/utils.go +++ b/tests/testcore/utils.go @@ -25,6 +25,7 @@ package testcore import ( + "encoding/base64" "fmt" "regexp" @@ -51,7 +52,7 @@ func GetFreeClusterNumber() int { return <-freeClusterNumbers } func PutFreeClusterNumber(i int) { freeClusterNumbers <- i } func RandomizeStr(id string) string { - return fmt.Sprintf("%v-%v", id, uuid.New()) + return fmt.Sprintf("%s-%.6s", id, base64.RawURLEncoding.EncodeToString(uuid.NewRandom())) } func DecodeString(t require.TestingT, pls *commonpb.Payloads) string { From 8cec9457c3fbc15e2f8228c9110b55dc518a37a7 Mon Sep 17 00:00:00 2001 From: David Reiss Date: Fri, 29 Nov 2024 20:03:21 -0500 Subject: [PATCH 13/21] rename dynamic config --- common/dynamicconfig/constants.go | 8 +++++--- service/frontend/service.go | 2 +- service/matching/config.go | 2 +- service/worker/deployment/fx.go | 2 +- tests/deployment_test.go | 2 +- tests/versioning_3_test.go | 2 +- tests/versioning_test.go | 2 +- 7 files changed, 11 insertions(+), 9 deletions(-) diff --git a/common/dynamicconfig/constants.go b/common/dynamicconfig/constants.go index 4ae8d8a0e17..14f4933fb7c 100644 --- a/common/dynamicconfig/constants.go +++ b/common/dynamicconfig/constants.go @@ -852,10 +852,12 @@ of Timeout and if no activity is seen even after that the connection is closed.` true, `FrontendEnableSchedules enables schedule-related RPCs in the frontend`, ) - FrontendEnableDeployments = NewNamespaceBoolSetting( - "frontend.enableDeployments", + EnableDeployments = NewNamespaceBoolSetting( + "system.enableDeployments", false, - `FrontendEnableDeployments enables deployment-related RPCs in the frontend`, + `EnableDeployments enables deployments (versioning v3) in all services, +including deployment-related RPCs in the frontend, deployment entity workflows in the worker, +and deployment interaction in matching and history.`, ) EnableNexus = NewGlobalBoolSetting( "system.enableNexus", diff --git a/service/frontend/service.go b/service/frontend/service.go index 77d03de1539..1377588fef3 100644 --- a/service/frontend/service.go +++ b/service/frontend/service.go @@ -309,7 +309,7 @@ func NewConfig( EnableSchedules: dynamicconfig.FrontendEnableSchedules.Get(dc), - EnableDeployments: dynamicconfig.FrontendEnableDeployments.Get(dc), + EnableDeployments: dynamicconfig.EnableDeployments.Get(dc), EnableBatcher: dynamicconfig.FrontendEnableBatcher.Get(dc), MaxConcurrentBatchOperation: dynamicconfig.FrontendMaxConcurrentBatchOperationPerNamespace.Get(dc), diff --git a/service/matching/config.go b/service/matching/config.go index fca1f13e5e3..b155bb03475 100644 --- a/service/matching/config.go +++ b/service/matching/config.go @@ -214,7 +214,7 @@ func NewConfig( TestDisableSyncMatch: dynamicconfig.TestMatchingDisableSyncMatch.Get(dc), LoadUserData: dynamicconfig.MatchingLoadUserData.Get(dc), HistoryMaxPageSize: dynamicconfig.MatchingHistoryMaxPageSize.Get(dc), - EnableDeployments: dynamicconfig.FrontendEnableDeployments.Get(dc), + EnableDeployments: dynamicconfig.EnableDeployments.Get(dc), MaxTaskQueuesInDeployment: dynamicconfig.MatchingMaxTaskQueuesInDeployment.Get(dc), RPS: dynamicconfig.MatchingRPS.Get(dc), OperatorRPSRatio: dynamicconfig.OperatorRPSRatio.Get(dc), diff --git a/service/worker/deployment/fx.go b/service/worker/deployment/fx.go index d052d0fb150..046cf8a4680 100644 --- a/service/worker/deployment/fx.go +++ b/service/worker/deployment/fx.go @@ -93,7 +93,7 @@ func NewResult( return fxResult{ Component: &workerComponent{ activityDeps: params, - enabledForNs: dynamicconfig.FrontendEnableDeployments.Get(dc), + enabledForNs: dynamicconfig.EnableDeployments.Get(dc), }, } } diff --git a/tests/deployment_test.go b/tests/deployment_test.go index 472818656ff..8683518c588 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -84,7 +84,7 @@ func TestDeploymentSuite(t *testing.T) { func (s *DeploymentSuite) SetupSuite() { s.setAssertions() dynamicConfigOverrides := map[dynamicconfig.Key]any{ - dynamicconfig.FrontendEnableDeployments.Key(): true, + dynamicconfig.EnableDeployments.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningDataAPIs.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningRuleAPIs.Key(): true, diff --git a/tests/versioning_3_test.go b/tests/versioning_3_test.go index cb032effc51..751f1944ec3 100644 --- a/tests/versioning_3_test.go +++ b/tests/versioning_3_test.go @@ -82,7 +82,7 @@ func TestVersioning3FunctionalSuite(t *testing.T) { func (s *Versioning3Suite) SetupSuite() { dynamicConfigOverrides := map[dynamicconfig.Key]any{ - dynamicconfig.FrontendEnableDeployments.Key(): true, + dynamicconfig.EnableDeployments.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, dynamicconfig.MatchingForwarderMaxChildrenPerNode.Key(): partitionTreeDegree, diff --git a/tests/versioning_test.go b/tests/versioning_test.go index 13a54649393..0f09fda385f 100644 --- a/tests/versioning_test.go +++ b/tests/versioning_test.go @@ -86,7 +86,7 @@ func TestVersioningFunctionalSuite(t *testing.T) { func (s *VersioningIntegSuite) SetupSuite() { dynamicConfigOverrides := map[dynamicconfig.Key]any{ - dynamicconfig.FrontendEnableDeployments.Key(): true, + dynamicconfig.EnableDeployments.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningDataAPIs.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningRuleAPIs.Key(): true, From f25176b463703fdca133c23d86a4dbc61aafa884 Mon Sep 17 00:00:00 2001 From: David Reiss Date: Fri, 29 Nov 2024 20:10:51 -0500 Subject: [PATCH 14/21] simplify code, rename constants --- .../matching/task_queue_partition_manager.go | 6 +- tests/deployment_test.go | 4 +- tests/versioning_3_test.go | 95 +++++++++---------- 3 files changed, 48 insertions(+), 57 deletions(-) diff --git a/service/matching/task_queue_partition_manager.go b/service/matching/task_queue_partition_manager.go index 20dd92b74ad..7c48e64b347 100644 --- a/service/matching/task_queue_partition_manager.go +++ b/service/matching/task_queue_partition_manager.go @@ -986,8 +986,6 @@ func (pm *taskQueuePartitionManagerImpl) getPerTypeUserData() (*persistencespb.T if err != nil { return nil, nil, err } - if perType, ok := userData.GetData().GetPerType()[int32(pm.Partition().TaskType())]; ok { - return perType, userDataChanged, nil - } - return nil, userDataChanged, nil + perType := userData.GetData().GetPerType()[int32(pm.Partition().TaskType())] + return perType, userDataChanged, nil } diff --git a/tests/deployment_test.go b/tests/deployment_test.go index 8683518c588..f2b0a9fdab9 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -207,9 +207,7 @@ func (s *DeploymentSuite) TestDescribeDeployment_RegisterTaskQueue_ConcurrentPol for p := 0; p < 4; p++ { for i := 0; i < 3; i++ { tq := &taskqueuepb.TaskQueue{Name: root.TaskQueue().NormalPartition(p).RpcName(), Kind: enumspb.TASK_QUEUE_KIND_NORMAL} - go func() { - s.pollFromDeployment(ctx, tq, workerDeployment) - }() + s.pollFromDeployment(ctx, tq, workerDeployment) } } diff --git a/tests/versioning_3_test.go b/tests/versioning_3_test.go index 751f1944ec3..aca0efef3dc 100644 --- a/tests/versioning_3_test.go +++ b/tests/versioning_3_test.go @@ -55,20 +55,15 @@ import ( "google.golang.org/protobuf/types/known/durationpb" ) -var ( - tqTypeWf = enumspb.TASK_QUEUE_TYPE_WORKFLOW - tqTypeAct = enumspb.TASK_QUEUE_TYPE_ACTIVITY - unspecified = enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED - pinned = enumspb.VERSIONING_BEHAVIOR_PINNED - unpinned = enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE - unpinnedOverride = &workflow.VersioningOverride{Behavior: unpinned} - minPollTime = common.MinLongPollTimeout + time.Millisecond*200 +const ( + tqTypeWf = enumspb.TASK_QUEUE_TYPE_WORKFLOW + tqTypeAct = enumspb.TASK_QUEUE_TYPE_ACTIVITY + vbUnspecified = enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED + vbPinned = enumspb.VERSIONING_BEHAVIOR_PINNED + vbUnpinned = enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE + ver3MinPollTime = common.MinLongPollTimeout + time.Millisecond*200 ) -func pinnedOverride(d *deploymentpb.Deployment) *workflow.VersioningOverride { - return &workflow.VersioningOverride{Behavior: pinned, Deployment: d} -} - type Versioning3Suite struct { // override suite.Suite.Assertions with require.Assertions; this means that s.NotNil(nil) will stop the test, // not merely log an error @@ -113,13 +108,11 @@ func (s *Versioning3Suite) TestPinnedTask_NoProperPoller() { tv := testvars.New(s) other := tv.WithBuildId("other") - go func() { - s.idlePollWorkflow(other, true, minPollTime, "other deployment should not receive pinned task") - }() + go s.idlePollWorkflow(other, true, ver3MinPollTime, "other deployment should not receive pinned task") s.waitForDeploymentDataPropagation(other, tqTypeWf) - s.startWorkflow(tv, pinnedOverride(tv.Deployment())) - s.idlePollWorkflow(tv, false, minPollTime, "unversioned worker should not receive pinned task") + s.startWorkflow(tv, makePinnedOverride(tv.Deployment())) + s.idlePollWorkflow(tv, false, ver3MinPollTime, "unversioned worker should not receive pinned task") }) } @@ -127,9 +120,7 @@ func (s *Versioning3Suite) TestUnpinnedTask_NonCurrentDeployment() { s.RunTestWithMatchingBehavior( func() { tv := testvars.New(s) - go func() { - s.idlePollWorkflow(tv, true, minPollTime, "non-current versioned poller should not receive unpinned task") - }() + go s.idlePollWorkflow(tv, true, ver3MinPollTime, "non-current versioned poller should not receive unpinned task") s.waitForDeploymentDataPropagation(tv, tqTypeWf) s.startWorkflow(tv, nil) @@ -150,7 +141,7 @@ func (s *Versioning3Suite) TestUnpinnedTask_OldDeployment() { s.idlePollWorkflow( tv.WithBuildId("older"), true, - minPollTime, + ver3MinPollTime, "old deployment should not receive unpinned task", ) }, @@ -184,7 +175,7 @@ func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { s.pollWftAndHandle(tv, false, wftCompleted, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - return respondWftWithActivities(tv, sticky, unpinned, "5"), nil + return respondWftWithActivities(tv, sticky, vbUnpinned, "5"), nil }) s.waitForDeploymentDataPropagation(tv, tqTypeWf) @@ -196,24 +187,24 @@ func (s *Versioning3Suite) testWorkflowWithPinnedOverride(sticky bool) { }) s.waitForDeploymentDataPropagation(tv, tqTypeAct) - override := pinnedOverride(tv.Deployment()) + override := makePinnedOverride(tv.Deployment()) we := s.startWorkflow(tv, override) <-wftCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + s.verifyWorkflowVersioning(tv, vbUnpinned, tv.Deployment(), override, nil) if sticky { s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) } <-actCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + s.verifyWorkflowVersioning(tv, vbUnpinned, tv.Deployment(), override, nil) s.pollWftAndHandle(tv, sticky, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - return respondCompleteWorkflow(tv, unpinned), nil + return respondCompleteWorkflow(tv, vbUnpinned), nil }) - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), override, nil) + s.verifyWorkflowVersioning(tv, vbUnpinned, tv.Deployment(), override, nil) } func (s *Versioning3Suite) TestUnpinnedWorkflow_Sticky() { @@ -244,8 +235,8 @@ func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { s.pollWftAndHandle(tv, false, wftCompleted, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - s.verifyWorkflowVersioning(tv, unspecified, nil, nil, transitionTo(d)) - return respondWftWithActivities(tv, sticky, unpinned, "5"), nil + s.verifyWorkflowVersioning(tv, vbUnspecified, nil, nil, transitionTo(d)) + return respondWftWithActivities(tv, sticky, vbUnpinned, "5"), nil }) s.waitForDeploymentDataPropagation(tv, tqTypeWf) @@ -262,20 +253,20 @@ func (s *Versioning3Suite) testUnpinnedWorkflow(sticky bool) { we := s.startWorkflow(tv, nil) <-wftCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) + s.verifyWorkflowVersioning(tv, vbUnpinned, tv.Deployment(), nil, nil) if sticky { s.verifyWorkflowStickyQueue(we, tv.StickyTaskQueue()) } <-actCompleted - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) + s.verifyWorkflowVersioning(tv, vbUnpinned, tv.Deployment(), nil, nil) s.pollWftAndHandle(tv, sticky, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - return respondCompleteWorkflow(tv, unpinned), nil + return respondCompleteWorkflow(tv, vbUnpinned), nil }) - s.verifyWorkflowVersioning(tv, unpinned, tv.Deployment(), nil, nil) + s.verifyWorkflowVersioning(tv, vbUnpinned, tv.Deployment(), nil, nil) } func (s *Versioning3Suite) TestTransitionFromWft_Sticky() { @@ -305,10 +296,10 @@ func (s *Versioning3Suite) testTransitionFromWft(sticky bool) { s.pollWftAndHandle(tvA, false, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - s.verifyWorkflowVersioning(tvA, unspecified, nil, nil, transitionTo(dA)) - return respondWftWithActivities(tvA, sticky, unpinned, "5"), nil + s.verifyWorkflowVersioning(tvA, vbUnspecified, nil, nil, transitionTo(dA)) + return respondWftWithActivities(tvA, sticky, vbUnpinned, "5"), nil }) - s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + s.verifyWorkflowVersioning(tvA, vbUnpinned, dA, nil, nil) if sticky { s.verifyWorkflowStickyQueue(we, tvA.StickyTaskQueue()) } @@ -318,7 +309,7 @@ func (s *Versioning3Suite) testTransitionFromWft(sticky bool) { s.NotNil(task) return respondActivity(), nil }) - s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + s.verifyWorkflowVersioning(tvA, vbUnpinned, dA, nil, nil) // Set B as the current deployment s.updateTaskQueueDeploymentData(tvB, 0, tqTypeWf, tqTypeAct) @@ -326,10 +317,10 @@ func (s *Versioning3Suite) testTransitionFromWft(sticky bool) { s.pollWftAndHandle(tvB, false, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, transitionTo(dB)) - return respondCompleteWorkflow(tvB, unpinned), nil + s.verifyWorkflowVersioning(tvA, vbUnpinned, dA, nil, transitionTo(dB)) + return respondCompleteWorkflow(tvB, vbUnpinned), nil }) - s.verifyWorkflowVersioning(tvB, unpinned, dB, nil, nil) + s.verifyWorkflowVersioning(tvB, vbUnpinned, dB, nil, nil) } func (s *Versioning3Suite) TestTransitionFromActivity_Sticky() { @@ -369,10 +360,10 @@ func (s *Versioning3Suite) testTransitionFromActivity(sticky bool) { s.pollWftAndHandle(tvA, false, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - s.verifyWorkflowVersioning(tvA, unspecified, nil, nil, transitionTo(dA)) - return respondWftWithActivities(tvA, sticky, unpinned, "5", "6", "7", "8"), nil + s.verifyWorkflowVersioning(tvA, vbUnspecified, nil, nil, transitionTo(dA)) + return respondWftWithActivities(tvA, sticky, vbUnpinned, "5", "6", "7", "8"), nil }) - s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + s.verifyWorkflowVersioning(tvA, vbUnpinned, dA, nil, nil) if sticky { s.verifyWorkflowStickyQueue(we, tvA.StickyTaskQueue()) } @@ -391,17 +382,17 @@ func (s *Versioning3Suite) testTransitionFromActivity(sticky bool) { s.NotNil(task) return respondActivity(), nil }) - s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, nil) + s.verifyWorkflowVersioning(tvA, vbUnpinned, dA, nil, nil) // The transition should create a new WFT to be sent to dB. Poller responds with empty wft complete. s.pollWftAndHandle(tvB, false, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - s.verifyWorkflowVersioning(tvA, unpinned, dA, nil, transitionTo(dB)) + s.verifyWorkflowVersioning(tvA, vbUnpinned, dA, nil, transitionTo(dB)) transitionCompleted.Store(true) - return respondEmptyWft(tvB, sticky, unpinned), nil + return respondEmptyWft(tvB, sticky, vbUnpinned), nil }) - s.verifyWorkflowVersioning(tvB, unpinned, dB, nil, nil) + s.verifyWorkflowVersioning(tvB, vbUnpinned, dB, nil, nil) if sticky { s.verifyWorkflowStickyQueue(we, tvB.StickyTaskQueue()) } @@ -409,9 +400,9 @@ func (s *Versioning3Suite) testTransitionFromActivity(sticky bool) { s.pollWftAndHandle(tvB, false, nil, func(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowservice.RespondWorkflowTaskCompletedRequest, error) { s.NotNil(task) - return respondCompleteWorkflow(tvB, unpinned), nil + return respondCompleteWorkflow(tvB, vbUnpinned), nil }) - s.verifyWorkflowVersioning(tvB, unpinned, dB, nil, nil) + s.verifyWorkflowVersioning(tvB, vbUnpinned, dB, nil, nil) } func transitionTo(d *deploymentpb.Deployment) *workflow.DeploymentTransition { @@ -769,7 +760,7 @@ func (s *Versioning3Suite) warmUpSticky( s.Fail("sticky task is not expected") return nil, nil }, - taskpoller.WithTimeout(minPollTime), + taskpoller.WithTimeout(ver3MinPollTime), ) } @@ -826,3 +817,7 @@ func (s *Versioning3Suite) waitForDeploymentDataPropagation( return len(remaining) == 0 }, 10*time.Second, 100*time.Millisecond) } + +func makePinnedOverride(d *deploymentpb.Deployment) *workflow.VersioningOverride { + return &workflow.VersioningOverride{Behavior: vbPinned, Deployment: d} +} From 7ab569beab71cad4615b703ab9ccd5890c1090c5 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 17:14:17 -0800 Subject: [PATCH 15/21] child workflow no override test works, with override poller is not getting tasks --- common/testing/taskpoller/taskpoller.go | 1 - tests/child_workflow_test.go | 44 ++++++++++++++----------- 2 files changed, 25 insertions(+), 20 deletions(-) diff --git a/common/testing/taskpoller/taskpoller.go b/common/testing/taskpoller/taskpoller.go index 3330688ab88..8b77a9370c8 100644 --- a/common/testing/taskpoller/taskpoller.go +++ b/common/testing/taskpoller/taskpoller.go @@ -233,7 +233,6 @@ func (p *workflowTaskPoller) pollTask( } var events []*historypb.HistoryEvent - fmt.Printf("\n\n\nCARLY DEBUG resp: %+v\n\n\n", resp) history := resp.History if history == nil { return nil, errors.New("history is nil") diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index c2abc91730b..340f9b8a021 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -98,26 +98,32 @@ func (s *ChildWorkflowSuite) checkDescribeWorkflowAfterOverride( } func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.VersioningOverride) { - tv := testvars.New(s.T()) var overrideDeployment *deploymentpb.Deployment var overrideBehavior enumspb.VersioningBehavior if override != nil { overrideDeployment = override.GetDeployment() overrideBehavior = override.GetBehavior() } - parentID := tv.String("functional-child-workflow-test-parent") - childID := tv.String("functional-child-workflow-test-child") - grandchildID := tv.String("functional-child-workflow-test-grandchild") - wtParent := tv.String("functional-child-workflow-test-parent-type") - wtChild := tv.String("functional-child-workflow-test-child-type") - wtGrandchild := tv.String("functional-child-workflow-test-grandchild-type") - tlParent := tv.String("functional-child-workflow-test-parent-taskqueue") - tlChild := tv.String("functional-child-workflow-test-child-taskqueue") - tlGrandchild := tv.String("functional-child-workflow-test-grandchild-taskqueue") - identity := tv.String("worker1") - saName := tv.String("CustomKeywordField") + parentTV := testvars.New(s.T()) + childTV := testvars.New(s.T()) + grandchildTV := testvars.New(s.T()) + + parentID := parentTV.String("functional-child-workflow-test-parent") + childID := childTV.String("functional-child-workflow-test-child") + grandchildID := grandchildTV.String("functional-child-workflow-test-grandchild") + wtParent := parentTV.String("functional-child-workflow-test-parent-type") + wtChild := childTV.String("functional-child-workflow-test-child-type") + wtGrandchild := grandchildTV.String("functional-child-workflow-test-grandchild-type") + tlParent := parentTV.String("functional-child-workflow-test-parent-taskqueue") + tlChild := childTV.String("functional-child-workflow-test-child-taskqueue") + tlGrandchild := grandchildTV.String("functional-child-workflow-test-grandchild-taskqueue") + identity := "worker1" + saName := "CustomKeywordField" // Uncomment this line to test with mapper. // saName = "AliasForCustomKeywordField" + parentTV = parentTV.WithTaskQueue(tlParent) + childTV = childTV.WithTaskQueue(tlChild) + grandchildTV = grandchildTV.WithTaskQueue(tlGrandchild) parentWorkflowType := &commonpb.WorkflowType{Name: wtParent} childWorkflowType := &commonpb.WorkflowType{Name: wtChild} @@ -354,7 +360,7 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver pollerGrandchild := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) // taskQueueGrandchild // Make first workflow task to start child execution - _, err := pollerParent.PollAndHandleWorkflowTask(tv, wtHandlerParent) + _, err := pollerParent.PollAndHandleWorkflowTask(parentTV, wtHandlerParent) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(childExecutionStarted) @@ -365,12 +371,12 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.Nil(parentStartedEventAttrs.GetRootWorkflowExecution()) // Process ChildExecution Started event and Process Child Execution and complete it - _, err = pollerParent.PollAndHandleWorkflowTask(tv, wtHandlerParent) + _, err = pollerParent.PollAndHandleWorkflowTask(parentTV, wtHandlerParent) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) // Process Child workflow to start grandchild execution - _, err = pollerChild.PollAndHandleWorkflowTask(tv, wtHandlerChild) + _, err = pollerChild.PollAndHandleWorkflowTask(childTV, wtHandlerChild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.NotNil(childStartedEventFromParent) @@ -409,12 +415,12 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.checkDescribeWorkflowAfterOverride(&commonpb.WorkflowExecution{WorkflowId: childID, RunId: childRunID}, override) // Process GrandchildExecution Started event and Process Grandchild Execution and complete it - _, err = pollerChild.PollAndHandleWorkflowTask(tv, wtHandlerChild) + _, err = pollerChild.PollAndHandleWorkflowTask(childTV, wtHandlerChild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) // Process Grandchild workflow - _, err = pollerGrandchild.PollAndHandleWorkflowTask(tv, wtHandlerGrandchild) + _, err = pollerGrandchild.PollAndHandleWorkflowTask(grandchildTV, wtHandlerGrandchild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(grandchildComplete) @@ -433,13 +439,13 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.ProtoEqual(override, grandchildStartedEventAttrs.GetVersioningOverride()) // Process GrandchildExecution completed event and complete child execution - _, err = pollerChild.PollAndHandleWorkflowTask(tv, wtHandlerChild) + _, err = pollerChild.PollAndHandleWorkflowTask(childTV, wtHandlerChild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(childComplete) // Process ChildExecution completed event and complete parent execution - _, err = pollerParent.PollAndHandleWorkflowTask(tv, wtHandlerParent) + _, err = pollerParent.PollAndHandleWorkflowTask(parentTV, wtHandlerParent) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.NotNil(childCompletedEventFromParent) From 0524e3bfa467f1a7becdd0af20f9edbd6e9528f3 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 17:51:28 -0800 Subject: [PATCH 16/21] push --- common/testing/testvars/test_vars.go | 13 ++++++++ tests/child_workflow_test.go | 44 +++++++++++++++++++++++++++- tests/deployment_test.go | 2 +- 3 files changed, 57 insertions(+), 2 deletions(-) diff --git a/common/testing/testvars/test_vars.go b/common/testing/testvars/test_vars.go index 56e0d77c847..9dcae0c0a21 100644 --- a/common/testing/testvars/test_vars.go +++ b/common/testing/testvars/test_vars.go @@ -25,6 +25,7 @@ package testvars import ( + deploymentpb "go.temporal.io/api/deployment/v1" "strings" "sync" "time" @@ -199,6 +200,18 @@ func (tv *TestVars) WithTaskQueue(taskQueue string, key ...string) *TestVars { return tv.cloneSet("task_queue", key, taskQueue) } +func (tv *TestVars) WithBuildId(buildId string, key ...string) *TestVars { + return tv.cloneSet("build_id", key, buildId) +} + +func (tv *TestVars) Deployment(key ...string) *deploymentpb.Deployment { + //revive:disable-next-line:unchecked-type-assertion + return &deploymentpb.Deployment{ + SeriesName: tv.DeploymentSeries(key...), + BuildId: tv.BuildId(key...), + } +} + func (tv *TestVars) StickyTaskQueue(key ...string) *taskqueuepb.TaskQueue { return &taskqueuepb.TaskQueue{ Name: tv.getOrCreate("sticky_task_queue", key).(string), diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index 340f9b8a021..83959216c6d 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -59,6 +59,34 @@ type ChildWorkflowSuite struct { testcore.FunctionalSuite } +//func (s *ChildWorkflowSuite) SetupSuite() { +// dynamicConfigOverrides := map[dynamicconfig.Key]any{ +// dynamicconfig.FrontendEnableDeployments.Key(): true, +// dynamicconfig.WorkerEnableDeployment.Key(): true, +// dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, +// dynamicconfig.MatchingForwarderMaxChildrenPerNode.Key(): partitionTreeDegree, +// +// // Make sure we don't hit the rate limiter in tests +// dynamicconfig.FrontendGlobalNamespaceNamespaceReplicationInducingAPIsRPS.Key(): 1000, +// dynamicconfig.FrontendMaxNamespaceNamespaceReplicationInducingAPIsBurstRatioPerInstance.Key(): 1, +// dynamicconfig.FrontendNamespaceReplicationInducingAPIsRPS.Key(): 1000, +// +// // this is overridden for tests using RunTestWithMatchingBehavior +// dynamicconfig.MatchingNumTaskqueueReadPartitions.Key(): 4, +// dynamicconfig.MatchingNumTaskqueueWritePartitions.Key(): 4, +// } +// s.SetDynamicConfigOverrides(dynamicConfigOverrides) +// s.FunctionalTestBase.SetupSuite("testdata/es_cluster.yaml") +//} +// +//func (s *ChildWorkflowSuite) TearDownSuite() { +// s.FunctionalTestBase.TearDownSuite() +//} +// +//func (s *ChildWorkflowSuite) SetupTest() { +// s.FunctionalTestBase.SetupTest() +//} + func TestChildWorkflowSuite(t *testing.T) { t.Parallel() suite.Run(t, new(ChildWorkflowSuite)) @@ -100,9 +128,15 @@ func (s *ChildWorkflowSuite) checkDescribeWorkflowAfterOverride( func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.VersioningOverride) { var overrideDeployment *deploymentpb.Deployment var overrideBehavior enumspb.VersioningBehavior + var versionCap *commonpb.WorkerVersionCapabilities if override != nil { overrideDeployment = override.GetDeployment() overrideBehavior = override.GetBehavior() + versionCap = &commonpb.WorkerVersionCapabilities{ + BuildId: overrideDeployment.GetBuildId(), + UseVersioning: true, + DeploymentSeriesName: overrideDeployment.GetSeriesName(), + } } parentTV := testvars.New(s.T()) childTV := testvars.New(s.T()) @@ -129,6 +163,12 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver childWorkflowType := &commonpb.WorkflowType{Name: wtChild} grandchildWorkflowType := &commonpb.WorkflowType{Name: wtGrandchild} + if overrideDeployment != nil { + parentTV = parentTV.WithBuildId(overrideDeployment.GetBuildId()) + childTV = childTV.WithBuildId(overrideDeployment.GetBuildId()) + grandchildTV = grandchildTV.WithBuildId(overrideDeployment.GetBuildId()) + } + taskQueueParent := &taskqueuepb.TaskQueue{Name: tlParent, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} taskQueueChild := &taskqueuepb.TaskQueue{Name: tlChild, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} taskQueueGrandchild := &taskqueuepb.TaskQueue{Name: tlGrandchild, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} @@ -360,7 +400,9 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver pollerGrandchild := taskpoller.New(s.T(), s.FrontendClient(), s.Namespace()) // taskQueueGrandchild // Make first workflow task to start child execution - _, err := pollerParent.PollAndHandleWorkflowTask(parentTV, wtHandlerParent) + _, err := pollerParent.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{WorkerVersionCapabilities: versionCap}, + ).HandleTask(parentTV, wtHandlerParent) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(childExecutionStarted) diff --git a/tests/deployment_test.go b/tests/deployment_test.go index c6e100e6a70..c292fb71a34 100644 --- a/tests/deployment_test.go +++ b/tests/deployment_test.go @@ -1041,7 +1041,7 @@ func (s *DeploymentSuite) TestUpdateWorkflowExecutionOptions_ChildWorkflowWithSD tqName := "test-tq-child-override" // create deployment so that GetDeploymentReachability doesn't error - s.createDeploymentAndWaitForExist(deploymentA, &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}) + s.createDeploymentAndWaitForExist(context.Background(), deploymentA, &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}) // define parent and child worfklows parentStarted := make(chan struct{}, 1) From 1ea10d7bed5d7d436ce27d69b0ad50fbc96b9dc6 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 17:53:18 -0800 Subject: [PATCH 17/21] Revert "Make ChildWorkflowExecution inherit versioning override of parent (#6911)" This reverts commit 74864793db7fb9476e860242a7a19698600aace9. --- service/history/transfer_queue_active_task_executor.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/service/history/transfer_queue_active_task_executor.go b/service/history/transfer_queue_active_task_executor.go index 283c6925f3a..58acf466cb4 100644 --- a/service/history/transfer_queue_active_task_executor.go +++ b/service/history/transfer_queue_active_task_executor.go @@ -854,7 +854,6 @@ func (t *transferQueueActiveTaskExecutor) processStartChildExecution( rootExecutionInfo, inheritedBuildId, initiatedEvent.GetUserMetadata(), - mutableState.GetExecutionInfo().GetVersioningInfo().GetVersioningOverride(), ) if err != nil { t.logger.Debug("Failed to start child workflow execution", tag.Error(err)) @@ -1345,7 +1344,6 @@ func (t *transferQueueActiveTaskExecutor) startWorkflow( rootExecutionInfo *workflowspb.RootExecutionInfo, inheritedBuildId string, userMetadata *sdkpb.UserMetadata, - inheritedOverride *workflowpb.VersioningOverride, ) (string, *clockspb.VectorClock, error) { request := common.CreateHistoryStartWorkflowRequest( task.TargetNamespaceID, @@ -1368,7 +1366,6 @@ func (t *transferQueueActiveTaskExecutor) startWorkflow( Memo: attributes.Memo, SearchAttributes: attributes.SearchAttributes, UserMetadata: userMetadata, - VersioningOverride: inheritedOverride, }, &workflowspb.ParentExecutionInfo{ NamespaceId: task.NamespaceID, From 5b92ab50a163d7dbada8babbed5dde8b8cb79b25 Mon Sep 17 00:00:00 2001 From: ShahabT Date: Fri, 29 Nov 2024 18:04:57 -0800 Subject: [PATCH 18/21] revert RandomizeStr change --- tests/testcore/utils.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/testcore/utils.go b/tests/testcore/utils.go index bd0fae1c14f..38a6a24ac24 100644 --- a/tests/testcore/utils.go +++ b/tests/testcore/utils.go @@ -25,7 +25,6 @@ package testcore import ( - "encoding/base64" "fmt" "regexp" @@ -52,7 +51,7 @@ func GetFreeClusterNumber() int { return <-freeClusterNumbers } func PutFreeClusterNumber(i int) { freeClusterNumbers <- i } func RandomizeStr(id string) string { - return fmt.Sprintf("%s-%.6s", id, base64.RawURLEncoding.EncodeToString(uuid.NewRandom())) + return fmt.Sprintf("%v-%v", id, uuid.New()) } func DecodeString(t require.TestingT, pls *commonpb.Payloads) string { From db2b3ea04128411b7089c69866600c2954a86196 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 18:09:04 -0800 Subject: [PATCH 19/21] fix nil pointer --- tests/child_workflow_test.go | 49 ++++++++++++++++++------------------ 1 file changed, 25 insertions(+), 24 deletions(-) diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index 83959216c6d..b5f11a60f13 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -27,6 +27,7 @@ package tests import ( "context" "fmt" + "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/testing/taskpoller" "go.temporal.io/server/common/testing/testvars" "sort" @@ -59,30 +60,30 @@ type ChildWorkflowSuite struct { testcore.FunctionalSuite } -//func (s *ChildWorkflowSuite) SetupSuite() { -// dynamicConfigOverrides := map[dynamicconfig.Key]any{ -// dynamicconfig.FrontendEnableDeployments.Key(): true, -// dynamicconfig.WorkerEnableDeployment.Key(): true, -// dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, -// dynamicconfig.MatchingForwarderMaxChildrenPerNode.Key(): partitionTreeDegree, -// -// // Make sure we don't hit the rate limiter in tests -// dynamicconfig.FrontendGlobalNamespaceNamespaceReplicationInducingAPIsRPS.Key(): 1000, -// dynamicconfig.FrontendMaxNamespaceNamespaceReplicationInducingAPIsBurstRatioPerInstance.Key(): 1, -// dynamicconfig.FrontendNamespaceReplicationInducingAPIsRPS.Key(): 1000, -// -// // this is overridden for tests using RunTestWithMatchingBehavior -// dynamicconfig.MatchingNumTaskqueueReadPartitions.Key(): 4, -// dynamicconfig.MatchingNumTaskqueueWritePartitions.Key(): 4, -// } -// s.SetDynamicConfigOverrides(dynamicConfigOverrides) -// s.FunctionalTestBase.SetupSuite("testdata/es_cluster.yaml") -//} -// -//func (s *ChildWorkflowSuite) TearDownSuite() { -// s.FunctionalTestBase.TearDownSuite() -//} -// +func (s *ChildWorkflowSuite) SetupSuite() { + dynamicConfigOverrides := map[dynamicconfig.Key]any{ + dynamicconfig.FrontendEnableDeployments.Key(): true, + dynamicconfig.WorkerEnableDeployment.Key(): true, + dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, + dynamicconfig.MatchingForwarderMaxChildrenPerNode.Key(): partitionTreeDegree, + + // Make sure we don't hit the rate limiter in tests + dynamicconfig.FrontendGlobalNamespaceNamespaceReplicationInducingAPIsRPS.Key(): 1000, + dynamicconfig.FrontendMaxNamespaceNamespaceReplicationInducingAPIsBurstRatioPerInstance.Key(): 1, + dynamicconfig.FrontendNamespaceReplicationInducingAPIsRPS.Key(): 1000, + + // this is overridden for tests using RunTestWithMatchingBehavior + dynamicconfig.MatchingNumTaskqueueReadPartitions.Key(): 4, + dynamicconfig.MatchingNumTaskqueueWritePartitions.Key(): 4, + } + s.SetDynamicConfigOverrides(dynamicConfigOverrides) + s.FunctionalTestBase.SetupSuite("testdata/es_cluster.yaml") +} + +func (s *ChildWorkflowSuite) TearDownSuite() { + s.FunctionalTestBase.TearDownSuite() +} + //func (s *ChildWorkflowSuite) SetupTest() { // s.FunctionalTestBase.SetupTest() //} From b34617b584724277e26560880ea00a72977ef22a Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 18:29:05 -0800 Subject: [PATCH 20/21] Revert "Revert "Make ChildWorkflowExecution inherit versioning override of parent (#6911)"" This reverts commit 1ea10d7bed5d7d436ce27d69b0ad50fbc96b9dc6. --- service/history/transfer_queue_active_task_executor.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/service/history/transfer_queue_active_task_executor.go b/service/history/transfer_queue_active_task_executor.go index 58acf466cb4..283c6925f3a 100644 --- a/service/history/transfer_queue_active_task_executor.go +++ b/service/history/transfer_queue_active_task_executor.go @@ -854,6 +854,7 @@ func (t *transferQueueActiveTaskExecutor) processStartChildExecution( rootExecutionInfo, inheritedBuildId, initiatedEvent.GetUserMetadata(), + mutableState.GetExecutionInfo().GetVersioningInfo().GetVersioningOverride(), ) if err != nil { t.logger.Debug("Failed to start child workflow execution", tag.Error(err)) @@ -1344,6 +1345,7 @@ func (t *transferQueueActiveTaskExecutor) startWorkflow( rootExecutionInfo *workflowspb.RootExecutionInfo, inheritedBuildId string, userMetadata *sdkpb.UserMetadata, + inheritedOverride *workflowpb.VersioningOverride, ) (string, *clockspb.VectorClock, error) { request := common.CreateHistoryStartWorkflowRequest( task.TargetNamespaceID, @@ -1366,6 +1368,7 @@ func (t *transferQueueActiveTaskExecutor) startWorkflow( Memo: attributes.Memo, SearchAttributes: attributes.SearchAttributes, UserMetadata: userMetadata, + VersioningOverride: inheritedOverride, }, &workflowspb.ParentExecutionInfo{ NamespaceId: task.NamespaceID, From d64a51b27b86dfce054d9662c0fac3c74b480410 Mon Sep 17 00:00:00 2001 From: Carly de Frondeville Date: Fri, 29 Nov 2024 18:31:17 -0800 Subject: [PATCH 21/21] test child workflow inherit --- common/testing/testvars/test_vars.go | 13 ------------ tests/child_workflow_test.go | 31 +++++++++++++++++----------- 2 files changed, 19 insertions(+), 25 deletions(-) diff --git a/common/testing/testvars/test_vars.go b/common/testing/testvars/test_vars.go index 180a0865e7d..1e4d6728a2d 100644 --- a/common/testing/testvars/test_vars.go +++ b/common/testing/testvars/test_vars.go @@ -25,7 +25,6 @@ package testvars import ( - deploymentpb "go.temporal.io/api/deployment/v1" "strings" "sync" "time" @@ -213,18 +212,6 @@ func (tv *TestVars) WithTaskQueue(taskQueue string, key ...string) *TestVars { return tv.cloneSet("task_queue", key, taskQueue) } -func (tv *TestVars) WithBuildId(buildId string, key ...string) *TestVars { - return tv.cloneSet("build_id", key, buildId) -} - -func (tv *TestVars) Deployment(key ...string) *deploymentpb.Deployment { - //revive:disable-next-line:unchecked-type-assertion - return &deploymentpb.Deployment{ - SeriesName: tv.DeploymentSeries(key...), - BuildId: tv.BuildId(key...), - } -} - func (tv *TestVars) StickyTaskQueue(key ...string) *taskqueuepb.TaskQueue { return &taskqueuepb.TaskQueue{ Name: tv.getOrCreate("sticky_task_queue", key).(string), diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index b5f11a60f13..5d12d796899 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -62,8 +62,7 @@ type ChildWorkflowSuite struct { func (s *ChildWorkflowSuite) SetupSuite() { dynamicConfigOverrides := map[dynamicconfig.Key]any{ - dynamicconfig.FrontendEnableDeployments.Key(): true, - dynamicconfig.WorkerEnableDeployment.Key(): true, + dynamicconfig.EnableDeployments.Key(): true, dynamicconfig.FrontendEnableWorkerVersioningWorkflowAPIs.Key(): true, dynamicconfig.MatchingForwarderMaxChildrenPerNode.Key(): partitionTreeDegree, @@ -84,10 +83,6 @@ func (s *ChildWorkflowSuite) TearDownSuite() { s.FunctionalTestBase.TearDownSuite() } -//func (s *ChildWorkflowSuite) SetupTest() { -// s.FunctionalTestBase.SetupTest() -//} - func TestChildWorkflowSuite(t *testing.T) { t.Parallel() suite.Run(t, new(ChildWorkflowSuite)) @@ -414,12 +409,16 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.Nil(parentStartedEventAttrs.GetRootWorkflowExecution()) // Process ChildExecution Started event and Process Child Execution and complete it - _, err = pollerParent.PollAndHandleWorkflowTask(parentTV, wtHandlerParent) + _, err = pollerParent.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{WorkerVersionCapabilities: versionCap}, + ).HandleTask(parentTV, wtHandlerParent) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) // Process Child workflow to start grandchild execution - _, err = pollerChild.PollAndHandleWorkflowTask(childTV, wtHandlerChild) + _, err = pollerChild.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{WorkerVersionCapabilities: versionCap}, + ).HandleTask(childTV, wtHandlerChild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.NotNil(childStartedEventFromParent) @@ -458,12 +457,16 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.checkDescribeWorkflowAfterOverride(&commonpb.WorkflowExecution{WorkflowId: childID, RunId: childRunID}, override) // Process GrandchildExecution Started event and Process Grandchild Execution and complete it - _, err = pollerChild.PollAndHandleWorkflowTask(childTV, wtHandlerChild) + _, err = pollerChild.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{WorkerVersionCapabilities: versionCap}, + ).HandleTask(childTV, wtHandlerChild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) // Process Grandchild workflow - _, err = pollerGrandchild.PollAndHandleWorkflowTask(grandchildTV, wtHandlerGrandchild) + _, err = pollerGrandchild.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{WorkerVersionCapabilities: versionCap}, + ).HandleTask(grandchildTV, wtHandlerGrandchild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(grandchildComplete) @@ -482,13 +485,17 @@ func (s *ChildWorkflowSuite) testChildWorkflowExecution(override *workflowpb.Ver s.ProtoEqual(override, grandchildStartedEventAttrs.GetVersioningOverride()) // Process GrandchildExecution completed event and complete child execution - _, err = pollerChild.PollAndHandleWorkflowTask(childTV, wtHandlerChild) + _, err = pollerChild.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{WorkerVersionCapabilities: versionCap}, + ).HandleTask(childTV, wtHandlerChild) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.True(childComplete) // Process ChildExecution completed event and complete parent execution - _, err = pollerParent.PollAndHandleWorkflowTask(parentTV, wtHandlerParent) + _, err = pollerParent.PollWorkflowTask( + &workflowservice.PollWorkflowTaskQueueRequest{WorkerVersionCapabilities: versionCap}, + ).HandleTask(parentTV, wtHandlerParent) s.Logger.Info("PollAndHandleWorkflowTask", tag.Error(err)) s.NoError(err) s.NotNil(childCompletedEventFromParent)